View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.client;
21  
22  import java.io.IOException;
23  import java.io.InterruptedIOException;
24  import java.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.Collection;
27  import java.util.HashMap;
28  import java.util.Iterator;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.concurrent.ConcurrentHashMap;
32  import java.util.concurrent.ConcurrentMap;
33  import java.util.concurrent.ConcurrentSkipListMap;
34  import java.util.concurrent.ExecutorService;
35  import java.util.concurrent.RejectedExecutionException;
36  import java.util.concurrent.atomic.AtomicBoolean;
37  import java.util.concurrent.atomic.AtomicInteger;
38  import java.util.concurrent.atomic.AtomicLong;
39  
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.hbase.DoNotRetryIOException;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HRegionInfo;
46  import org.apache.hadoop.hbase.HRegionLocation;
47  import org.apache.hadoop.hbase.ServerName;
48  import org.apache.hadoop.hbase.TableName;
49  import org.apache.hadoop.hbase.util.Bytes;
50  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
51  import org.apache.hadoop.hbase.util.Pair;
52  import org.cloudera.htrace.Trace;
53  
54  /**
55   * This class  allows a continuous flow of requests. It's written to be compatible with a
56   * synchronous caller such as HTable.
57   * <p>
58   * The caller sends a buffer of operation, by calling submit. This class extract from this list
59   * the operations it can send, i.e. the operations that are on region that are not considered
60   * as busy. The process is asynchronous, i.e. it returns immediately when if has finished to
61   * iterate on the list. If, and only if, the maximum number of current task is reached, the call
62   * to submit will block.
63   * </p>
64   * <p>
65   * The class manages internally the retries.
66   * </p>
67   * <p>
68   * The class includes an error marker: it allows to know if an operation has failed or not, and
69   * to get the exception details, i.e. the full list of throwables for each attempt. This marker
70   * is here to help the backward compatibility in HTable. In most (new) cases, it should be
71   * managed by the callbacks.
72   * </p>
73   * <p>
74   * A callback is available, in order to: <list>
75   * <li>Get the result of the operation (failure or success)</li>
76   * <li>When an operation fails but could be retried, allows or not to retry</li>
77   * <li>When an operation fails for good (can't be retried or already retried the maximum number
78   * time), register the error or not.
79   * </list>
80   * <p>
81   * This class is not thread safe externally; only one thread should submit operations at a time.
82   * Internally, the class is thread safe enough to manage simultaneously new submission and results
83   * arising from older operations.
84   * </p>
85   * <p>
86   * Internally, this class works with {@link Row}, this mean it could be theoretically used for
87   * gets as well.
88   * </p>
89   */
90  class AsyncProcess<CResult> {
91    private static final Log LOG = LogFactory.getLog(AsyncProcess.class);
92    protected static final AtomicLong COUNTER = new AtomicLong();
93    protected final long id;
94    private final int startLogErrorsCnt;
95    protected final HConnection hConnection;
96    protected final TableName tableName;
97    protected final ExecutorService pool;
98    protected final AsyncProcessCallback<CResult> callback;
99    protected final BatchErrors errors = new BatchErrors();
100   protected final AtomicBoolean hasError = new AtomicBoolean(false);
101   protected final AtomicLong tasksSent = new AtomicLong(0);
102   protected final AtomicLong tasksDone = new AtomicLong(0);
103   protected final AtomicLong retriesCnt = new AtomicLong(0);
104   protected final ConcurrentMap<byte[], AtomicInteger> taskCounterPerRegion =
105       new ConcurrentSkipListMap<byte[], AtomicInteger>(Bytes.BYTES_COMPARATOR);
106   protected final ConcurrentMap<ServerName, AtomicInteger> taskCounterPerServer =
107       new ConcurrentHashMap<ServerName, AtomicInteger>();
108 
109   /**
110    * The number of tasks simultaneously executed on the cluster.
111    */
112   protected final int maxTotalConcurrentTasks;
113 
114   /**
115    * The number of tasks we run in parallel on a single region.
116    * With 1 (the default) , we ensure that the ordering of the queries is respected: we don't start
117    * a set of operations on a region before the previous one is done. As well, this limits
118    * the pressure we put on the region server.
119    */
120   protected final int maxConcurrentTasksPerRegion;
121 
122   /**
123    * The number of task simultaneously executed on a single region server.
124    */
125   protected final int maxConcurrentTasksPerServer;
126   protected final long pause;
127   protected int numTries;
128   protected int serverTrackerTimeout;
129   protected RpcRetryingCallerFactory rpcCallerFactory;
130 
131 
132   /**
133    * This interface allows to keep the interface of the previous synchronous interface, that uses
134    * an array of object to return the result.
135    * <p/>
136    * This interface allows the caller to specify the behavior on errors: <list>
137    * <li>If we have not yet reach the maximum number of retries, the user can nevertheless
138    * specify if this specific operation should be retried or not.
139    * </li>
140    * <li>If an operation fails (i.e. is not retried or fails after all retries), the user can
141    * specify is we should mark this AsyncProcess as in error or not.
142    * </li>
143    * </list>
144    */
145   interface AsyncProcessCallback<CResult> {
146 
147     /**
148      * Called on success. originalIndex holds the index in the action list.
149      */
150     void success(int originalIndex, byte[] region, Row row, CResult result);
151 
152     /**
153      * called on failure, if we don't retry (i.e. called once per failed operation).
154      *
155      * @return true if we should store the error and tag this async process as being in error.
156      *         false if the failure of this operation can be safely ignored, and does not require
157      *         the current process to be stopped without proceeding with the other operations in
158      *         the queue.
159      */
160     boolean failure(int originalIndex, byte[] region, Row row, Throwable t);
161 
162     /**
163      * Called on a failure we plan to retry. This allows the user to stop retrying. Will be
164      * called multiple times for a single action if it fails multiple times.
165      *
166      * @return false if we should retry, true otherwise.
167      */
168     boolean retriableFailure(int originalIndex, Row row, byte[] region, Throwable exception);
169   }
170 
171   private static class BatchErrors {
172     private final List<Throwable> throwables = new ArrayList<Throwable>();
173     private final List<Row> actions = new ArrayList<Row>();
174     private final List<String> addresses = new ArrayList<String>();
175 
176     public synchronized void add(Throwable ex, Row row, HRegionLocation location) {
177       if (row == null){
178         throw new IllegalArgumentException("row cannot be null. location=" + location);
179       }
180 
181       throwables.add(ex);
182       actions.add(row);
183       addresses.add(location != null ? location.getServerName().toString() : "null location");
184     }
185 
186     private synchronized RetriesExhaustedWithDetailsException makeException() {
187       return new RetriesExhaustedWithDetailsException(
188           new ArrayList<Throwable>(throwables),
189           new ArrayList<Row>(actions), new ArrayList<String>(addresses));
190     }
191 
192     public synchronized void clear() {
193       throwables.clear();
194       actions.clear();
195       addresses.clear();
196     }
197   }
198 
199   public AsyncProcess(HConnection hc, TableName tableName, ExecutorService pool,
200       AsyncProcessCallback<CResult> callback, Configuration conf,
201       RpcRetryingCallerFactory rpcCaller) {
202     if (hc == null){
203       throw new IllegalArgumentException("HConnection cannot be null.");
204     }
205 
206     this.hConnection = hc;
207     this.tableName = tableName;
208     this.pool = pool;
209     this.callback = callback;
210 
211     this.id = COUNTER.incrementAndGet();
212 
213     this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
214         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
215     this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
216         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
217 
218     this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
219       HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS);
220     this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
221           HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS);
222     this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
223           HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS);
224 
225     // A few failure is fine: region moved, then is not opened, then is overloaded. We try
226     //  to have an acceptable heuristic for the number of errors we don't log.
227     //  9 was chosen because we wait for 1s at this stage.
228     this.startLogErrorsCnt = conf.getInt("hbase.client.start.log.errors.counter", 9);
229 
230     if (this.maxTotalConcurrentTasks <= 0) {
231       throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
232     }
233     if (this.maxConcurrentTasksPerServer <= 0) {
234       throw new IllegalArgumentException("maxConcurrentTasksPerServer=" +
235           maxConcurrentTasksPerServer);
236     }
237     if (this.maxConcurrentTasksPerRegion <= 0) {
238       throw new IllegalArgumentException("maxConcurrentTasksPerRegion=" +
239           maxConcurrentTasksPerRegion);
240     }
241 
242     // Server tracker allows us to do faster, and yet useful (hopefully), retries.
243     // However, if we are too useful, we might fail very quickly due to retry count limit.
244     // To avoid this, we are going to cheat for now (see HBASE-7659), and calculate maximum
245     // retry time if normal retries were used. Then we will retry until this time runs out.
246     // If we keep hitting one server, the net effect will be the incremental backoff, and
247     // essentially the same number of retries as planned. If we have to do faster retries,
248     // we will do more retries in aggregate, but the user will be none the wiser.
249     this.serverTrackerTimeout = 0;
250     for (int i = 0; i < this.numTries; ++i) {
251       serverTrackerTimeout += ConnectionUtils.getPauseTime(this.pause, i);
252     }
253 
254 
255     this.rpcCallerFactory = rpcCaller;
256   }
257 
258   /**
259    * Extract from the rows list what we can submit. The rows we can not submit are kept in the
260    * list.
261    *
262    * @param rows - the submitted row. Modified by the method: we remove the rows we took.
263    * @param atLeastOne true if we should submit at least a subset.
264    */
265   public void submit(List<? extends Row> rows, boolean atLeastOne) throws InterruptedIOException {
266     if (rows.isEmpty()) {
267       return;
268     }
269 
270     // This looks like we are keying by region but HRegionLocation has a comparator that compares
271     // on the server portion only (hostname + port) so this Map collects regions by server.
272     Map<HRegionLocation, MultiAction<Row>> actionsByServer =
273       new HashMap<HRegionLocation, MultiAction<Row>>();
274     List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(rows.size());
275 
276     long currentTaskCnt = tasksDone.get();
277     boolean alreadyLooped = false;
278 
279     do {
280       if (alreadyLooped){
281         // if, for whatever reason, we looped, we want to be sure that something has changed.
282         waitForNextTaskDone(currentTaskCnt);
283         currentTaskCnt = tasksDone.get();
284       } else {
285         alreadyLooped = true;
286       }
287 
288       // Wait until there is at least one slot for a new task.
289       waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1);
290 
291       // Remember the previous decisions about regions or region servers we put in the
292       //  final multi.
293       Map<Long, Boolean> regionIncluded = new HashMap<Long, Boolean>();
294       Map<ServerName, Boolean> serverIncluded = new HashMap<ServerName, Boolean>();
295 
296       int posInList = -1;
297       Iterator<? extends Row> it = rows.iterator();
298       while (it.hasNext()) {
299         Row r = it.next();
300         HRegionLocation loc = findDestLocation(r, posInList);
301 
302         if (loc == null) { // loc is null if there is an error such as meta not available.
303           it.remove();
304         } else if (canTakeOperation(loc, regionIncluded, serverIncluded)) {
305           Action<Row> action = new Action<Row>(r, ++posInList);
306           retainedActions.add(action);
307           addAction(loc, action, actionsByServer);
308           it.remove();
309         }
310       }
311 
312     } while (retainedActions.isEmpty() && atLeastOne && !hasError());
313 
314     HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
315     sendMultiAction(retainedActions, actionsByServer, 1, errorsByServer);
316   }
317 
318   /**
319    * Group the actions per region server.
320    *
321    * @param loc - the destination. Must not be null.
322    * @param action - the action to add to the multiaction
323    * @param actionsByServer the multiaction per server
324    */
325   private void addAction(HRegionLocation loc, Action<Row> action, Map<HRegionLocation,
326       MultiAction<Row>> actionsByServer) {
327     final byte[] regionName = loc.getRegionInfo().getRegionName();
328     MultiAction<Row> multiAction = actionsByServer.get(loc);
329     if (multiAction == null) {
330       multiAction = new MultiAction<Row>();
331       actionsByServer.put(loc, multiAction);
332     }
333 
334     multiAction.add(regionName, action);
335   }
336 
337   /**
338    * Find the destination.
339    *
340    * @param row          the row
341    * @param posInList    the position in the list
342    * @return the destination. Null if we couldn't find it.
343    */
344   private HRegionLocation findDestLocation(Row row, int posInList) {
345     if (row == null) throw new IllegalArgumentException("#" + id + ", row cannot be null");
346     HRegionLocation loc = null;
347     IOException locationException = null;
348     try {
349       loc = hConnection.locateRegion(this.tableName, row.getRow());
350       if (loc == null) {
351         locationException = new IOException("#" + id + ", no location found, aborting submit for" +
352             " tableName=" + tableName +
353             " rowkey=" + Arrays.toString(row.getRow()));
354       }
355     } catch (IOException e) {
356       locationException = e;
357     }
358     if (locationException != null) {
359       // There are multiple retries in locateRegion already. No need to add new.
360       // We can't continue with this row, hence it's the last retry.
361       manageError(posInList, row, false, locationException, null);
362       return null;
363     }
364 
365     return loc;
366   }
367 
368   /**
369    * Check if we should send new operations to this region or region server.
370    * We're taking into account the past decision; if we have already accepted
371    * operation on a given region, we accept all operations for this region.
372    *
373    * @param loc; the region and the server name we want to use.
374    * @return true if this region is considered as busy.
375    */
376   protected boolean canTakeOperation(HRegionLocation loc,
377                                      Map<Long, Boolean> regionsIncluded,
378                                      Map<ServerName, Boolean> serversIncluded) {
379     long regionId = loc.getRegionInfo().getRegionId();
380     Boolean regionPrevious = regionsIncluded.get(regionId);
381 
382     if (regionPrevious != null) {
383       // We already know what to do with this region.
384       return regionPrevious;
385     }
386 
387     Boolean serverPrevious = serversIncluded.get(loc.getServerName());
388     if (Boolean.FALSE.equals(serverPrevious)) {
389       // It's a new region, on a region server that we have already excluded.
390       regionsIncluded.put(regionId, Boolean.FALSE);
391       return false;
392     }
393 
394     AtomicInteger regionCnt = taskCounterPerRegion.get(loc.getRegionInfo().getRegionName());
395     if (regionCnt != null && regionCnt.get() >= maxConcurrentTasksPerRegion) {
396       // Too many tasks on this region already.
397       regionsIncluded.put(regionId, Boolean.FALSE);
398       return false;
399     }
400 
401     if (serverPrevious == null) {
402       // The region is ok, but we need to decide for this region server.
403       int newServers = 0; // number of servers we're going to contact so far
404       for (Map.Entry<ServerName, Boolean> kv : serversIncluded.entrySet()) {
405         if (kv.getValue()) {
406           newServers++;
407         }
408       }
409 
410       // Do we have too many total tasks already?
411       boolean ok = (newServers + getCurrentTasksCount()) < maxTotalConcurrentTasks;
412 
413       if (ok) {
414         // If the total is fine, is it ok for this individual server?
415         AtomicInteger serverCnt = taskCounterPerServer.get(loc.getServerName());
416         ok = (serverCnt == null || serverCnt.get() < maxConcurrentTasksPerServer);
417       }
418 
419       if (!ok) {
420         regionsIncluded.put(regionId, Boolean.FALSE);
421         serversIncluded.put(loc.getServerName(), Boolean.FALSE);
422         return false;
423       }
424 
425       serversIncluded.put(loc.getServerName(), Boolean.TRUE);
426     } else {
427       assert serverPrevious.equals(Boolean.TRUE);
428     }
429 
430     regionsIncluded.put(regionId, Boolean.TRUE);
431 
432     return true;
433   }
434 
435   /**
436    * Submit immediately the list of rows, whatever the server status. Kept for backward
437    * compatibility: it allows to be used with the batch interface that return an array of objects.
438    *
439    * @param rows the list of rows.
440    */
441   public void submitAll(List<? extends Row> rows) {
442     List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
443 
444     // The position will be used by the processBatch to match the object array returned.
445     int posInList = -1;
446     for (Row r : rows) {
447       posInList++;
448       Action<Row> action = new Action<Row>(r, posInList);
449       actions.add(action);
450     }
451     HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
452     submit(actions, actions, 1, errorsByServer);
453   }
454 
455 
456   /**
457    * Group a list of actions per region servers, and send them. The created MultiActions are
458    * added to the inProgress list. Does not take into account the region/server load.
459    *
460    * @param initialActions - the full list of the actions in progress
461    * @param currentActions - the list of row to submit
462    * @param numAttempt - the current numAttempt (first attempt is 1)
463    */
464   private void submit(List<Action<Row>> initialActions,
465                       List<Action<Row>> currentActions, int numAttempt,
466                       final HConnectionManager.ServerErrorTracker errorsByServer) {
467 
468     if (numAttempt > 1){
469       retriesCnt.incrementAndGet();
470     }
471 
472     // group per location => regions server
473     final Map<HRegionLocation, MultiAction<Row>> actionsByServer =
474         new HashMap<HRegionLocation, MultiAction<Row>>();
475 
476     for (Action<Row> action : currentActions) {
477       HRegionLocation loc = findDestLocation(action.getAction(), action.getOriginalIndex());
478       if (loc != null) {
479         addAction(loc, action, actionsByServer);
480       }
481     }
482 
483     if (!actionsByServer.isEmpty()) {
484       sendMultiAction(initialActions, actionsByServer, numAttempt, errorsByServer);
485     }
486   }
487 
488   /**
489    * Send a multi action structure to the servers, after a delay depending on the attempt
490    * number. Asynchronous.
491    *
492    * @param initialActions  the list of the actions, flat.
493    * @param actionsByServer the actions structured by regions
494    * @param numAttempt      the attempt number.
495    */
496   public void sendMultiAction(final List<Action<Row>> initialActions,
497                               Map<HRegionLocation, MultiAction<Row>> actionsByServer,
498                               final int numAttempt,
499                               final HConnectionManager.ServerErrorTracker errorsByServer) {
500     // Send the queries and add them to the inProgress list
501     // This iteration is by server (the HRegionLocation comparator is by server portion only).
502     for (Map.Entry<HRegionLocation, MultiAction<Row>> e : actionsByServer.entrySet()) {
503       final HRegionLocation loc = e.getKey();
504       final MultiAction<Row> multiAction = e.getValue();
505       incTaskCounters(multiAction.getRegions(), loc.getServerName());
506       Runnable runnable = Trace.wrap("AsyncProcess.sendMultiAction", new Runnable() {
507         @Override
508         public void run() {
509           MultiResponse res;
510           try {
511             MultiServerCallable<Row> callable = createCallable(loc, multiAction);
512             try {
513               res = createCaller(callable).callWithoutRetries(callable);
514             } catch (IOException e) {
515               // The service itself failed . It may be an error coming from the communication
516               //   layer, but, as well, a functional error raised by the server.
517               receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, e,
518                   errorsByServer);
519               return;
520             } catch (Throwable t) {
521               // This should not happen. Let's log & retry anyway.
522               LOG.error("#" + id + ", Caught throwable while calling. This is unexpected." +
523                   " Retrying. Server is " + loc.getServerName() + ", tableName=" + tableName, t);
524               receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, t,
525                   errorsByServer);
526               return;
527             }
528 
529             // Nominal case: we received an answer from the server, and it's not an exception.
530             receiveMultiAction(initialActions, multiAction, loc, res, numAttempt, errorsByServer);
531 
532           } finally {
533             decTaskCounters(multiAction.getRegions(), loc.getServerName());
534           }
535         }
536       });
537 
538       try {
539         this.pool.submit(runnable);
540       } catch (RejectedExecutionException ree) {
541         // This should never happen. But as the pool is provided by the end user, let's secure
542         //  this a little.
543         decTaskCounters(multiAction.getRegions(), loc.getServerName());
544         LOG.warn("#" + id + ", the task was rejected by the pool. This is unexpected." +
545             " Server is " + loc.getServerName(), ree);
546         // We're likely to fail again, but this will increment the attempt counter, so it will
547         //  finish.
548         receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, ree, errorsByServer);
549       }
550     }
551   }
552 
553   /**
554    * Create a callable. Isolated to be easily overridden in the tests.
555    */
556   protected MultiServerCallable<Row> createCallable(final HRegionLocation location,
557       final MultiAction<Row> multi) {
558     return new MultiServerCallable<Row>(hConnection, tableName, location, multi);
559   }
560 
561   /**
562    * For tests.
563    * @param callable: used in tests.
564    * @return Returns a caller.
565    */
566   protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
567     return rpcCallerFactory.<MultiResponse> newCaller();
568   }
569 
570   /**
571    * Check that we can retry acts accordingly: logs, set the error status, call the callbacks.
572    *
573    * @param originalIndex the position in the list sent
574    * @param row           the row
575    * @param canRetry      if false, we won't retry whatever the settings.
576    * @param throwable     the throwable, if any (can be null)
577    * @param location      the location, if any (can be null)
578    * @return true if the action can be retried, false otherwise.
579    */
580   private boolean manageError(int originalIndex, Row row, boolean canRetry,
581                               Throwable throwable, HRegionLocation location) {
582     if (canRetry && throwable != null && throwable instanceof DoNotRetryIOException) {
583       canRetry = false;
584     }
585 
586     byte[] region = null;
587     if (canRetry && callback != null) {
588       region = location == null ? null : location.getRegionInfo().getEncodedNameAsBytes();
589       canRetry = callback.retriableFailure(originalIndex, row, region, throwable);
590     }
591 
592     if (!canRetry) {
593       if (callback != null) {
594         if (region == null && location != null) {
595           region = location.getRegionInfo().getEncodedNameAsBytes();
596         }
597         callback.failure(originalIndex, region, row, throwable);
598       }
599       errors.add(throwable, row, location);
600       this.hasError.set(true);
601     }
602 
603     return canRetry;
604   }
605 
606   /**
607    * Resubmit all the actions from this multiaction after a failure.
608    *
609    * @param initialActions the full initial action list
610    * @param rsActions  the actions still to do from the initial list
611    * @param location   the destination
612    * @param numAttempt the number of attempts so far
613    * @param t the throwable (if any) that caused the resubmit
614    */
615   private void receiveGlobalFailure(List<Action<Row>> initialActions, MultiAction<Row> rsActions,
616                                     HRegionLocation location, int numAttempt, Throwable t,
617                                     HConnectionManager.ServerErrorTracker errorsByServer) {
618     // Do not use the exception for updating cache because it might be coming from
619     // any of the regions in the MultiAction.
620     hConnection.updateCachedLocations(tableName,
621       rsActions.actions.values().iterator().next().get(0).getAction().getRow(), null, location);
622     errorsByServer.reportServerError(location);
623 
624     List<Action<Row>> toReplay = new ArrayList<Action<Row>>(initialActions.size());
625     for (Map.Entry<byte[], List<Action<Row>>> e : rsActions.actions.entrySet()) {
626       for (Action<Row> action : e.getValue()) {
627         if (manageError(action.getOriginalIndex(), action.getAction(), true, t, location)) {
628           toReplay.add(action);
629         }
630       }
631     }
632 
633     logAndResubmit(initialActions, location, toReplay, numAttempt, rsActions.size(),
634         t, errorsByServer);
635   }
636 
637   /**
638    * Log as many info as possible, and, if there is something to replay, submit it again after
639    *  a back off sleep.
640    */
641   private void logAndResubmit(List<Action<Row>> initialActions, HRegionLocation oldLocation,
642                               List<Action<Row>> toReplay, int numAttempt, int failureCount,
643                               Throwable throwable,
644                               HConnectionManager.ServerErrorTracker errorsByServer) {
645     if (toReplay.isEmpty()) {
646       // it's either a success or a last failure
647       if (failureCount != 0) {
648         // We have a failure but nothing to retry. We're done, it's a final failure..
649         LOG.warn(createLog(numAttempt, failureCount, toReplay.size(),
650             oldLocation.getServerName(), throwable, -1, false,
651             errorsByServer.getStartTrackingTime()));
652       } else if (numAttempt > startLogErrorsCnt + 1) {
653         // The operation was successful, but needed several attempts. Let's log this.
654         LOG.info(createLog(numAttempt, failureCount, 0,
655             oldLocation.getServerName(), throwable, -1, false,
656             errorsByServer.getStartTrackingTime()));
657       }
658       return;
659     }
660 
661     // We have something to replay. We're going to sleep a little before.
662 
663     // We have two contradicting needs here:
664     //  1) We want to get the new location after having slept, as it may change.
665     //  2) We want to take into account the location when calculating the sleep time.
666     // It should be possible to have some heuristics to take the right decision. Short term,
667     //  we go for one.
668     long backOffTime = errorsByServer.calculateBackoffTime(oldLocation, pause);
669 
670     if (numAttempt > startLogErrorsCnt) {
671       // We use this value to have some logs when we have multiple failures, but not too many
672       //  logs, as errors are to be expected when a region moves, splits and so on
673       LOG.info(createLog(numAttempt, failureCount, toReplay.size(),
674           oldLocation.getServerName(), throwable, backOffTime, true,
675           errorsByServer.getStartTrackingTime()));
676     }
677 
678     try {
679       Thread.sleep(backOffTime);
680     } catch (InterruptedException e) {
681       LOG.warn("#" + id + ", not sent: " + toReplay.size() + " operations, " + oldLocation, e);
682       Thread.interrupted();
683       return;
684     }
685 
686     submit(initialActions, toReplay, numAttempt + 1, errorsByServer);
687   }
688 
689   /**
690    * Called when we receive the result of a server query.
691    *
692    * @param initialActions - the whole action list
693    * @param multiAction    - the multiAction we sent
694    * @param location       - the location. It's used as a server name.
695    * @param responses      - the response, if any
696    * @param numAttempt     - the attempt
697    */
698   private void receiveMultiAction(List<Action<Row>> initialActions, MultiAction<Row> multiAction,
699                                   HRegionLocation location,
700                                   MultiResponse responses, int numAttempt,
701                                   HConnectionManager.ServerErrorTracker errorsByServer) {
702      assert responses != null;
703 
704     // Success or partial success
705     // Analyze detailed results. We can still have individual failures to be redo.
706     // two specific throwables are managed:
707     //  - DoNotRetryIOException: we continue to retry for other actions
708     //  - RegionMovedException: we update the cache with the new region location
709 
710     List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
711     Throwable throwable = null;
712     int failureCount = 0;
713     boolean canRetry = true;
714 
715     for (Map.Entry<byte[], List<Pair<Integer, Object>>> resultsForRS :
716         responses.getResults().entrySet()) {
717 
718       boolean regionFailureRegistered = false;
719       for (Pair<Integer, Object> regionResult : resultsForRS.getValue()) {
720         Object result = regionResult.getSecond();
721 
722         // Failure: retry if it's make sense else update the errors lists
723         if (result == null || result instanceof Throwable) {
724           throwable = (Throwable) result;
725           Action<Row> correspondingAction = initialActions.get(regionResult.getFirst());
726           Row row = correspondingAction.getAction();
727           failureCount++;
728           if (!regionFailureRegistered) { // We're doing this once per location.
729             regionFailureRegistered= true;
730             // The location here is used as a server name.
731             hConnection.updateCachedLocations(this.tableName, row.getRow(), result, location);
732             if (failureCount == 1) {
733               errorsByServer.reportServerError(location);
734               canRetry = errorsByServer.canRetryMore(numAttempt);
735             }
736           }
737 
738           if (manageError(correspondingAction.getOriginalIndex(), row, canRetry,
739               throwable, location)) {
740             toReplay.add(correspondingAction);
741           }
742         } else { // success
743           if (callback != null) {
744             int index = regionResult.getFirst();
745             Action<Row> correspondingAction = initialActions.get(index);
746             Row row = correspondingAction.getAction();
747             //noinspection unchecked
748             this.callback.success(index, resultsForRS.getKey(), row, (CResult) result);
749           }
750         }
751       }
752     }
753 
754     // The failures global to a region. We will use for multiAction we sent previously to find the
755     //   actions to replay.
756 
757     for (Map.Entry<byte[], Throwable> throwableEntry : responses.getExceptions().entrySet()) {
758       throwable = throwableEntry.getValue();
759       byte[] region =throwableEntry.getKey();
760       List<Action<Row>> actions = multiAction.actions.get(region);
761       if (actions == null || actions.isEmpty()) {
762         throw new IllegalStateException("Wrong response for the region: " +
763             HRegionInfo.encodeRegionName(region));
764       }
765 
766       if (failureCount == 0) {
767         errorsByServer.reportServerError(location);
768         canRetry = errorsByServer.canRetryMore(numAttempt);
769       }
770       hConnection.updateCachedLocations(this.tableName, actions.get(0).getAction().getRow(),
771           throwable, location);
772       failureCount += actions.size();
773 
774       for (Action<Row> action : actions) {
775         Row row = action.getAction();
776         if (manageError(action.getOriginalIndex(), row, canRetry, throwable, location)) {
777           toReplay.add(action);
778         }
779       }
780     }
781 
782     logAndResubmit(initialActions, location, toReplay, numAttempt, failureCount,
783         throwable, errorsByServer);
784   }
785 
786   private String createLog(int numAttempt, int failureCount, int replaySize, ServerName sn,
787                            Throwable error, long backOffTime, boolean willRetry, String startTime){
788     StringBuilder sb = new StringBuilder();
789 
790     sb.append("#").append(id).append(", table=").append(tableName).
791         append(", attempt=").append(numAttempt).append("/").append(numTries).append(" ");
792 
793     if (failureCount > 0 || error != null){
794       sb.append("failed ").append(failureCount).append(" ops").append(", last exception: ").
795           append(error == null ? "null" : error);
796     } else {
797       sb.append("SUCCEEDED");
798     }
799 
800     sb.append(" on ").append(sn);
801 
802     sb.append(", tracking started ").append(startTime);
803 
804     if (willRetry) {
805       sb.append(", retrying after ").append(backOffTime).append(" ms").
806           append(", replay ").append(replaySize).append(" ops.");
807     } else if (failureCount > 0) {
808       sb.append(" - FAILED, NOT RETRYING ANYMORE");
809     }
810 
811     return sb.toString();
812   }
813 
814   /**
815    * Waits for another task to finish.
816    * @param currentNumberOfTask - the number of task finished when calling the method.
817    */
818   protected void waitForNextTaskDone(long currentNumberOfTask) throws InterruptedIOException {
819     while (currentNumberOfTask == tasksDone.get()) {
820       try {
821         synchronized (this.tasksDone) {
822           this.tasksDone.wait(100);
823         }
824       } catch (InterruptedException e) {
825         throw new InterruptedIOException("#" + id + ", interrupted." +
826             " currentNumberOfTask=" + currentNumberOfTask +
827             ",  tableName=" + tableName + ", tasksDone=" + tasksDone.get());
828       }
829     }
830   }
831 
832   /**
833    * Wait until the async does not have more than max tasks in progress.
834    */
835   private void waitForMaximumCurrentTasks(int max) throws InterruptedIOException {
836     long lastLog = EnvironmentEdgeManager.currentTimeMillis();
837     long currentTasksDone = this.tasksDone.get();
838 
839     while ((tasksSent.get() - currentTasksDone) > max) {
840       long now = EnvironmentEdgeManager.currentTimeMillis();
841       if (now > lastLog + 10000) {
842         lastLog = now;
843         LOG.info("#" + id + ", waiting for some tasks to finish. Expected max="
844             + max + ", tasksSent=" + tasksSent.get() + ", tasksDone=" + tasksDone.get() +
845             ", currentTasksDone=" + currentTasksDone + ", retries=" + retriesCnt.get() +
846             " hasError=" + hasError() + ", tableName=" + tableName);
847       }
848       waitForNextTaskDone(currentTasksDone);
849       currentTasksDone = this.tasksDone.get();
850     }
851   }
852 
853   private long getCurrentTasksCount(){
854     return  tasksSent.get() - tasksDone.get();
855   }
856 
857   /**
858    * Wait until all tasks are executed, successfully or not.
859    */
860   public void waitUntilDone() throws InterruptedIOException {
861     waitForMaximumCurrentTasks(0);
862   }
863 
864 
865   public boolean hasError() {
866     return hasError.get();
867   }
868 
869   public List<? extends Row> getFailedOperations() {
870     return errors.actions;
871   }
872 
873   /**
874    * Clean the errors stacks. Should be called only when there are no actions in progress.
875    */
876   public void clearErrors() {
877     errors.clear();
878     hasError.set(false);
879   }
880 
881   public RetriesExhaustedWithDetailsException getErrors() {
882     return errors.makeException();
883   }
884 
885   /**
886    * increment the tasks counters for a given set of regions. MT safe.
887    */
888   protected void incTaskCounters(Collection<byte[]> regions, ServerName sn) {
889     tasksSent.incrementAndGet();
890 
891     AtomicInteger serverCnt = taskCounterPerServer.get(sn);
892     if (serverCnt == null) {
893       taskCounterPerServer.putIfAbsent(sn, new AtomicInteger());
894       serverCnt = taskCounterPerServer.get(sn);
895     }
896     serverCnt.incrementAndGet();
897 
898     for (byte[] regBytes : regions) {
899       AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
900       if (regionCnt == null) {
901         regionCnt = new AtomicInteger();
902         AtomicInteger oldCnt = taskCounterPerRegion.putIfAbsent(regBytes, regionCnt);
903         if (oldCnt != null) {
904           regionCnt = oldCnt;
905         }
906       }
907       regionCnt.incrementAndGet();
908     }
909   }
910 
911   /**
912    * Decrements the counters for a given region and the region server. MT Safe.
913    */
914   protected void decTaskCounters(Collection<byte[]> regions, ServerName sn) {
915     for (byte[] regBytes : regions) {
916       AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
917       regionCnt.decrementAndGet();
918     }
919 
920     taskCounterPerServer.get(sn).decrementAndGet();
921 
922     tasksDone.incrementAndGet();
923     synchronized (tasksDone) {
924       tasksDone.notifyAll();
925     }
926   }
927 
928   /**
929    * Creates the server error tracker to use inside process.
930    * Currently, to preserve the main assumption about current retries, and to work well with
931    * the retry-limit-based calculation, the calculation is local per Process object.
932    * We may benefit from connection-wide tracking of server errors.
933    * @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection
934    */
935   protected HConnectionManager.ServerErrorTracker createServerErrorTracker() {
936     return new HConnectionManager.ServerErrorTracker(this.serverTrackerTimeout, this.numTries);
937   }
938 }