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.Collection;
26  import java.util.Collections;
27  import java.util.Date;
28  import java.util.HashMap;
29  import java.util.Iterator;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.Set;
33  import java.util.concurrent.ConcurrentHashMap;
34  import java.util.concurrent.ConcurrentMap;
35  import java.util.concurrent.ConcurrentSkipListMap;
36  import java.util.concurrent.ExecutorService;
37  import java.util.concurrent.RejectedExecutionException;
38  import java.util.concurrent.TimeUnit;
39  import java.util.concurrent.atomic.AtomicInteger;
40  import java.util.concurrent.atomic.AtomicLong;
41  
42  import org.apache.commons.logging.Log;
43  import org.apache.commons.logging.LogFactory;
44  import org.apache.hadoop.hbase.classification.InterfaceAudience;
45  import org.apache.hadoop.conf.Configuration;
46  import org.apache.hadoop.hbase.DoNotRetryIOException;
47  import org.apache.hadoop.hbase.HConstants;
48  import org.apache.hadoop.hbase.HRegionInfo;
49  import org.apache.hadoop.hbase.HRegionLocation;
50  import org.apache.hadoop.hbase.RegionLocations;
51  import org.apache.hadoop.hbase.ServerName;
52  import org.apache.hadoop.hbase.TableName;
53  import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
54  import org.apache.hadoop.hbase.client.coprocessor.Batch;
55  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
56  import org.apache.hadoop.hbase.util.Bytes;
57  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
58  import org.apache.htrace.Trace;
59  
60  import com.google.common.annotations.VisibleForTesting;
61  
62  /**
63   * This class  allows a continuous flow of requests. It's written to be compatible with a
64   * synchronous caller such as HTable.
65   * <p>
66   * The caller sends a buffer of operation, by calling submit. This class extract from this list
67   * the operations it can send, i.e. the operations that are on region that are not considered
68   * as busy. The process is asynchronous, i.e. it returns immediately when if has finished to
69   * iterate on the list. If, and only if, the maximum number of current task is reached, the call
70   * to submit will block. Alternatively, the caller can call submitAll, in which case all the
71   * operations will be sent. Each call to submit returns a future-like object that can be used
72   * to track operation progress.
73   * </p>
74   * <p>
75   * The class manages internally the retries.
76   * </p>
77   * <p>
78   * The class can be constructed in regular mode, or "global error" mode. In global error mode,
79   * AP tracks errors across all calls (each "future" also has global view of all errors). That
80   * mode is necessary for backward compat with HTable behavior, where multiple submissions are
81   * made and the errors can propagate using any put/flush call, from previous calls.
82   * In "regular" mode, the errors are tracked inside the Future object that is returned.
83   * The results are always tracked inside the Future object and can be retrieved when the call
84   * has finished. Partial results can also be retrieved if some part of multi-request failed.
85   * </p>
86   * <p>
87   * This class is thread safe in regular mode; in global error code, submitting operations and
88   * retrieving errors from different threads may be not thread safe.
89   * Internally, the class is thread safe enough to manage simultaneously new submission and results
90   * arising from older operations.
91   * </p>
92   * <p>
93   * Internally, this class works with {@link Row}, this mean it could be theoretically used for
94   * gets as well.
95   * </p>
96   */
97  @InterfaceAudience.Private
98  class AsyncProcess {
99    protected static final Log LOG = LogFactory.getLog(AsyncProcess.class);
100   protected static final AtomicLong COUNTER = new AtomicLong();
101 
102   public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget";
103 
104   /**
105    * Configure the number of failures after which the client will start logging. A few failures
106    * is fine: region moved, then is not opened, then is overloaded. We try to have an acceptable
107    * heuristic for the number of errors we don't log. 9 was chosen because we wait for 1s at
108    * this stage.
109    */
110   public static final String START_LOG_ERRORS_AFTER_COUNT_KEY =
111       "hbase.client.start.log.errors.counter";
112   public static final int DEFAULT_START_LOG_ERRORS_AFTER_COUNT = 9;
113 
114   private final int thresholdToLogUndoneTaskDetails;
115   private static final String THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS =
116       "hbase.client.threshold.log.details";
117   private static final int DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS = 10;
118   private final int THRESHOLD_TO_LOG_REGION_DETAILS = 2;
119 
120   /**
121    * The context used to wait for results from one submit call.
122    * 1) If AsyncProcess is set to track errors globally, and not per call (for HTable puts),
123    *    then errors and failed operations in this object will reflect global errors.
124    * 2) If submit call is made with needResults false, results will not be saved.
125    *  */
126   public static interface AsyncRequestFuture {
127     public boolean hasError();
128     public RetriesExhaustedWithDetailsException getErrors();
129     public List<? extends Row> getFailedOperations();
130     public Object[] getResults() throws InterruptedIOException;
131     /** Wait until all tasks are executed, successfully or not. */
132     public void waitUntilDone() throws InterruptedIOException;
133   }
134 
135   /** Return value from a submit that didn't contain any requests. */
136   private static final AsyncRequestFuture NO_REQS_RESULT = new AsyncRequestFuture() {
137     public final Object[] result = new Object[0];
138     @Override
139     public boolean hasError() { return false; }
140     @Override
141     public RetriesExhaustedWithDetailsException getErrors() { return null; }
142     @Override
143     public List<? extends Row> getFailedOperations() { return null; }
144     @Override
145     public Object[] getResults() { return result; }
146     @Override
147     public void waitUntilDone() throws InterruptedIOException {}
148   };
149 
150   /** Sync point for calls to multiple replicas for the same user request (Get).
151    * Created and put in the results array (we assume replica calls require results) when
152    * the replica calls are launched. See results for details of this process.
153    * POJO, all fields are public. To modify them, the object itself is locked. */
154   private static class ReplicaResultState {
155     public ReplicaResultState(int callCount) {
156       this.callCount = callCount;
157     }
158 
159     /** Number of calls outstanding, or 0 if a call succeeded (even with others outstanding). */
160     int callCount;
161     /** Errors for which it is not decided whether we will report them to user. If one of the
162      * calls succeeds, we will discard the errors that may have happened in the other calls. */
163     BatchErrors replicaErrors = null;
164 
165     @Override
166     public String toString() {
167       return "[call count " + callCount + "; errors " + replicaErrors + "]";
168     }
169   }
170 
171 
172   // TODO: many of the fields should be made private
173   protected final long id;
174 
175   protected final ClusterConnection connection;
176   protected final RpcRetryingCallerFactory rpcCallerFactory;
177   protected final RpcControllerFactory rpcFactory;
178   protected final BatchErrors globalErrors;
179   protected final ExecutorService pool;
180 
181   protected final AtomicLong tasksInProgress = new AtomicLong(0);
182   protected final ConcurrentMap<byte[], AtomicInteger> taskCounterPerRegion =
183       new ConcurrentSkipListMap<byte[], AtomicInteger>(Bytes.BYTES_COMPARATOR);
184   protected final ConcurrentMap<ServerName, AtomicInteger> taskCounterPerServer =
185       new ConcurrentHashMap<ServerName, AtomicInteger>();
186 
187   // Start configuration settings.
188   private final int startLogErrorsCnt;
189 
190   /**
191    * The number of tasks simultaneously executed on the cluster.
192    */
193   protected final int maxTotalConcurrentTasks;
194 
195   /**
196    * The number of tasks we run in parallel on a single region.
197    * With 1 (the default) , we ensure that the ordering of the queries is respected: we don't start
198    * a set of operations on a region before the previous one is done. As well, this limits
199    * the pressure we put on the region server.
200    */
201   protected final int maxConcurrentTasksPerRegion;
202 
203   /**
204    * The number of task simultaneously executed on a single region server.
205    */
206   protected final int maxConcurrentTasksPerServer;
207   protected final long pause;
208   protected int numTries;
209   protected int serverTrackerTimeout;
210   protected int timeout;
211   protected long primaryCallTimeoutMicroseconds;
212   // End configuration settings.
213 
214   protected static class BatchErrors {
215     private final List<Throwable> throwables = new ArrayList<Throwable>();
216     private final List<Row> actions = new ArrayList<Row>();
217     private final List<String> addresses = new ArrayList<String>();
218 
219     public synchronized void add(Throwable ex, Row row, ServerName serverName) {
220       if (row == null){
221         throw new IllegalArgumentException("row cannot be null. location=" + serverName);
222       }
223 
224       throwables.add(ex);
225       actions.add(row);
226       addresses.add(serverName != null ? serverName.toString() : "null");
227     }
228 
229     public boolean hasErrors() {
230       return !throwables.isEmpty();
231     }
232 
233     private synchronized RetriesExhaustedWithDetailsException makeException() {
234       return new RetriesExhaustedWithDetailsException(
235           new ArrayList<Throwable>(throwables),
236           new ArrayList<Row>(actions), new ArrayList<String>(addresses));
237     }
238 
239     public synchronized void clear() {
240       throwables.clear();
241       actions.clear();
242       addresses.clear();
243     }
244 
245     public synchronized void merge(BatchErrors other) {
246       throwables.addAll(other.throwables);
247       actions.addAll(other.actions);
248       addresses.addAll(other.addresses);
249     }
250   }
251 
252   public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService pool,
253       RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors, RpcControllerFactory rpcFactory) {
254     if (hc == null) {
255       throw new IllegalArgumentException("HConnection cannot be null.");
256     }
257 
258     this.connection = hc;
259     this.pool = pool;
260     this.globalErrors = useGlobalErrors ? new BatchErrors() : null;
261 
262     this.id = COUNTER.incrementAndGet();
263 
264     this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
265         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
266     this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
267         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
268     this.timeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
269         HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
270     this.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000);
271 
272     this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
273       HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS);
274     this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
275           HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS);
276     this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
277           HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS);
278 
279     this.startLogErrorsCnt =
280         conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
281 
282     if (this.maxTotalConcurrentTasks <= 0) {
283       throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
284     }
285     if (this.maxConcurrentTasksPerServer <= 0) {
286       throw new IllegalArgumentException("maxConcurrentTasksPerServer=" +
287           maxConcurrentTasksPerServer);
288     }
289     if (this.maxConcurrentTasksPerRegion <= 0) {
290       throw new IllegalArgumentException("maxConcurrentTasksPerRegion=" +
291           maxConcurrentTasksPerRegion);
292     }
293 
294     // Server tracker allows us to do faster, and yet useful (hopefully), retries.
295     // However, if we are too useful, we might fail very quickly due to retry count limit.
296     // To avoid this, we are going to cheat for now (see HBASE-7659), and calculate maximum
297     // retry time if normal retries were used. Then we will retry until this time runs out.
298     // If we keep hitting one server, the net effect will be the incremental backoff, and
299     // essentially the same number of retries as planned. If we have to do faster retries,
300     // we will do more retries in aggregate, but the user will be none the wiser.
301     this.serverTrackerTimeout = 0;
302     for (int i = 0; i < this.numTries; ++i) {
303       serverTrackerTimeout += ConnectionUtils.getPauseTime(this.pause, i);
304     }
305 
306     this.rpcCallerFactory = rpcCaller;
307     this.rpcFactory = rpcFactory;
308     this.thresholdToLogUndoneTaskDetails =
309         conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS,
310             DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS);
311   }
312 
313   /**
314    * @return pool if non null, otherwise returns this.pool if non null, otherwise throws
315    *         RuntimeException
316    */
317   private ExecutorService getPool(ExecutorService pool) {
318     if (pool != null) return pool;
319     if (this.pool != null) return this.pool;
320     throw new RuntimeException("Neither AsyncProcess nor request have ExecutorService");
321   }
322 
323   /**
324    * See {@link #submit(ExecutorService, TableName, List, boolean, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback, boolean)}.
325    * Uses default ExecutorService for this AP (must have been created with one).
326    */
327   public <CResult> AsyncRequestFuture submit(TableName tableName, List<? extends Row> rows,
328       boolean atLeastOne, Batch.Callback<CResult> callback, boolean needResults)
329       throws InterruptedIOException {
330     return submit(null, tableName, rows, atLeastOne, callback, needResults);
331   }
332 
333   /**
334    * Extract from the rows list what we can submit. The rows we can not submit are kept in the
335    * list. Does not send requests to replicas (not currently used for anything other
336    * than streaming puts anyway).
337    *
338    * @param pool ExecutorService to use.
339    * @param tableName The table for which this request is needed.
340    * @param callback Batch callback. Only called on success (94 behavior).
341    * @param needResults Whether results are needed, or can be discarded.
342    * @param rows - the submitted row. Modified by the method: we remove the rows we took.
343    * @param atLeastOne true if we should submit at least a subset.
344    */
345   public <CResult> AsyncRequestFuture submit(ExecutorService pool, TableName tableName,
346       List<? extends Row> rows, boolean atLeastOne, Batch.Callback<CResult> callback,
347       boolean needResults) throws InterruptedIOException {
348     if (rows.isEmpty()) {
349       return NO_REQS_RESULT;
350     }
351 
352     Map<ServerName, MultiAction<Row>> actionsByServer =
353         new HashMap<ServerName, MultiAction<Row>>();
354     List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(rows.size());
355 
356     NonceGenerator ng = this.connection.getNonceGenerator();
357     long nonceGroup = ng.getNonceGroup(); // Currently, nonce group is per entire client.
358 
359     // Location errors that happen before we decide what requests to take.
360     List<Exception> locationErrors = null;
361     List<Integer> locationErrorRows = null;
362     do {
363       // Wait until there is at least one slot for a new task.
364       waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1, tableName.getNameAsString());
365 
366       // Remember the previous decisions about regions or region servers we put in the
367       //  final multi.
368       Map<Long, Boolean> regionIncluded = new HashMap<Long, Boolean>();
369       Map<ServerName, Boolean> serverIncluded = new HashMap<ServerName, Boolean>();
370 
371       int posInList = -1;
372       Iterator<? extends Row> it = rows.iterator();
373       while (it.hasNext()) {
374         Row r = it.next();
375         HRegionLocation loc;
376         try {
377           if (r == null) throw new IllegalArgumentException("#" + id + ", row cannot be null");
378           // Make sure we get 0-s replica.
379           RegionLocations locs = connection.locateRegion(
380               tableName, r.getRow(), true, true, RegionReplicaUtil.DEFAULT_REPLICA_ID);
381           if (locs == null || locs.isEmpty() || locs.getDefaultRegionLocation() == null) {
382             throw new IOException("#" + id + ", no location found, aborting submit for"
383                 + " tableName=" + tableName + " rowkey=" + Bytes.toStringBinary(r.getRow()));
384           }
385           loc = locs.getDefaultRegionLocation();
386         } catch (IOException ex) {
387           locationErrors = new ArrayList<Exception>();
388           locationErrorRows = new ArrayList<Integer>();
389           LOG.error("Failed to get region location ", ex);
390           // This action failed before creating ars. Retain it, but do not add to submit list.
391           // We will then add it to ars in an already-failed state.
392           retainedActions.add(new Action<Row>(r, ++posInList));
393           locationErrors.add(ex);
394           locationErrorRows.add(posInList);
395           it.remove();
396           break; // Backward compat: we stop considering actions on location error.
397         }
398 
399         if (canTakeOperation(loc, regionIncluded, serverIncluded)) {
400           Action<Row> action = new Action<Row>(r, ++posInList);
401           setNonce(ng, r, action);
402           retainedActions.add(action);
403           // TODO: replica-get is not supported on this path
404           byte[] regionName = loc.getRegionInfo().getRegionName();
405           addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
406           it.remove();
407         }
408       }
409     } while (retainedActions.isEmpty() && atLeastOne && (locationErrors == null));
410 
411     if (retainedActions.isEmpty()) return NO_REQS_RESULT;
412 
413     return submitMultiActions(tableName, retainedActions, nonceGroup, callback, null, needResults,
414       locationErrors, locationErrorRows, actionsByServer, pool);
415   }
416 
417   <CResult> AsyncRequestFuture submitMultiActions(TableName tableName,
418       List<Action<Row>> retainedActions, long nonceGroup, Batch.Callback<CResult> callback,
419       Object[] results, boolean needResults, List<Exception> locationErrors,
420       List<Integer> locationErrorRows, Map<ServerName, MultiAction<Row>> actionsByServer,
421       ExecutorService pool) {
422     AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(
423       tableName, retainedActions, nonceGroup, pool, callback, results, needResults);
424     // Add location errors if any
425     if (locationErrors != null) {
426       for (int i = 0; i < locationErrors.size(); ++i) {
427         int originalIndex = locationErrorRows.get(i);
428         Row row = retainedActions.get(originalIndex).getAction();
429         ars.manageError(originalIndex, row,
430           Retry.NO_LOCATION_PROBLEM, locationErrors.get(i), null);
431       }
432     }
433     ars.sendMultiAction(actionsByServer, 1, null, false);
434     return ars;
435   }
436 
437   /**
438    * Helper that is used when grouping the actions per region server.
439    *
440    * @param loc - the destination. Must not be null.
441    * @param action - the action to add to the multiaction
442    * @param actionsByServer the multiaction per server
443    * @param nonceGroup Nonce group.
444    */
445   private static void addAction(ServerName server, byte[] regionName, Action<Row> action,
446       Map<ServerName, MultiAction<Row>> actionsByServer, long nonceGroup) {
447     MultiAction<Row> multiAction = actionsByServer.get(server);
448     if (multiAction == null) {
449       multiAction = new MultiAction<Row>();
450       actionsByServer.put(server, multiAction);
451     }
452     if (action.hasNonce() && !multiAction.hasNonceGroup()) {
453       multiAction.setNonceGroup(nonceGroup);
454     }
455 
456     multiAction.add(regionName, action);
457   }
458 
459   /**
460    * Check if we should send new operations to this region or region server.
461    * We're taking into account the past decision; if we have already accepted
462    * operation on a given region, we accept all operations for this region.
463    *
464    * @param loc; the region and the server name we want to use.
465    * @return true if this region is considered as busy.
466    */
467   protected boolean canTakeOperation(HRegionLocation loc,
468                                      Map<Long, Boolean> regionsIncluded,
469                                      Map<ServerName, Boolean> serversIncluded) {
470     long regionId = loc.getRegionInfo().getRegionId();
471     Boolean regionPrevious = regionsIncluded.get(regionId);
472 
473     if (regionPrevious != null) {
474       // We already know what to do with this region.
475       return regionPrevious;
476     }
477 
478     Boolean serverPrevious = serversIncluded.get(loc.getServerName());
479     if (Boolean.FALSE.equals(serverPrevious)) {
480       // It's a new region, on a region server that we have already excluded.
481       regionsIncluded.put(regionId, Boolean.FALSE);
482       return false;
483     }
484 
485     AtomicInteger regionCnt = taskCounterPerRegion.get(loc.getRegionInfo().getRegionName());
486     if (regionCnt != null && regionCnt.get() >= maxConcurrentTasksPerRegion) {
487       // Too many tasks on this region already.
488       regionsIncluded.put(regionId, Boolean.FALSE);
489       return false;
490     }
491 
492     if (serverPrevious == null) {
493       // The region is ok, but we need to decide for this region server.
494       int newServers = 0; // number of servers we're going to contact so far
495       for (Map.Entry<ServerName, Boolean> kv : serversIncluded.entrySet()) {
496         if (kv.getValue()) {
497           newServers++;
498         }
499       }
500 
501       // Do we have too many total tasks already?
502       boolean ok = (newServers + tasksInProgress.get()) < maxTotalConcurrentTasks;
503 
504       if (ok) {
505         // If the total is fine, is it ok for this individual server?
506         AtomicInteger serverCnt = taskCounterPerServer.get(loc.getServerName());
507         ok = (serverCnt == null || serverCnt.get() < maxConcurrentTasksPerServer);
508       }
509 
510       if (!ok) {
511         regionsIncluded.put(regionId, Boolean.FALSE);
512         serversIncluded.put(loc.getServerName(), Boolean.FALSE);
513         return false;
514       }
515 
516       serversIncluded.put(loc.getServerName(), Boolean.TRUE);
517     } else {
518       assert serverPrevious.equals(Boolean.TRUE);
519     }
520 
521     regionsIncluded.put(regionId, Boolean.TRUE);
522 
523     return true;
524   }
525 
526   /**
527    * See {@link #submitAll(ExecutorService, TableName, List, org.apache.hadoop.hbase.client.coprocessor.Batch.Callback, Object[])}.
528    * Uses default ExecutorService for this AP (must have been created with one).
529    */
530   public <CResult> AsyncRequestFuture submitAll(TableName tableName,
531       List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results) {
532     return submitAll(null, tableName, rows, callback, results);
533   }
534 
535   /**
536    * Submit immediately the list of rows, whatever the server status. Kept for backward
537    * compatibility: it allows to be used with the batch interface that return an array of objects.
538    *
539    * @param pool ExecutorService to use.
540    * @param tableName name of the table for which the submission is made.
541    * @param rows the list of rows.
542    * @param callback the callback.
543    * @param results Optional array to return the results thru; backward compat.
544    */
545   public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
546       List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results) {
547     List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
548 
549     // The position will be used by the processBatch to match the object array returned.
550     int posInList = -1;
551     NonceGenerator ng = this.connection.getNonceGenerator();
552     for (Row r : rows) {
553       posInList++;
554       if (r instanceof Put) {
555         Put put = (Put) r;
556         if (put.isEmpty()) {
557           throw new IllegalArgumentException("No columns to insert for #" + (posInList+1)+ " item");
558         }
559       }
560       Action<Row> action = new Action<Row>(r, posInList);
561       setNonce(ng, r, action);
562       actions.add(action);
563     }
564     AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(
565         tableName, actions, ng.getNonceGroup(), getPool(pool), callback, results, results != null);
566     ars.groupAndSendMultiAction(actions, 1);
567     return ars;
568   }
569 
570   private static void setNonce(NonceGenerator ng, Row r, Action<Row> action) {
571     if (!(r instanceof Append) && !(r instanceof Increment)) return;
572     action.setNonce(ng.newNonce()); // Action handles NO_NONCE, so it's ok if ng is disabled.
573   }
574 
575   /**
576    * The context, and return value, for a single submit/submitAll call.
577    * Note on how this class (one AP submit) works. Initially, all requests are split into groups
578    * by server; request is sent to each server in parallel; the RPC calls are not async so a
579    * thread per server is used. Every time some actions fail, regions/locations might have
580    * changed, so we re-group them by server and region again and send these groups in parallel
581    * too. The result, in case of retries, is a "tree" of threads, with parent exiting after
582    * scheduling children. This is why lots of code doesn't require any synchronization.
583    */
584   protected class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
585 
586     /**
587      * Runnable (that can be submitted to thread pool) that waits for when it's time
588      * to issue replica calls, finds region replicas, groups the requests by replica and
589      * issues the calls (on separate threads, via sendMultiAction).
590      * This is done on a separate thread because we don't want to wait on user thread for
591      * our asynchronous call, and usually we have to wait before making replica calls.
592      */
593     private final class ReplicaCallIssuingRunnable implements Runnable {
594       private final long startTime;
595       private final List<Action<Row>> initialActions;
596 
597       public ReplicaCallIssuingRunnable(List<Action<Row>> initialActions, long startTime) {
598         this.initialActions = initialActions;
599         this.startTime = startTime;
600       }
601 
602       @Override
603       public void run() {
604         boolean done = false;
605         if (primaryCallTimeoutMicroseconds > 0) {
606           try {
607             done = waitUntilDone(startTime * 1000L + primaryCallTimeoutMicroseconds);
608           } catch (InterruptedException ex) {
609             LOG.error("Replica thread was interrupted - no replica calls: " + ex.getMessage());
610             return;
611           }
612         }
613         if (done) return; // Done within primary timeout
614         Map<ServerName, MultiAction<Row>> actionsByServer =
615             new HashMap<ServerName, MultiAction<Row>>();
616         List<Action<Row>> unknownLocActions = new ArrayList<Action<Row>>();
617         if (replicaGetIndices == null) {
618           for (int i = 0; i < results.length; ++i) {
619             addReplicaActions(i, actionsByServer, unknownLocActions);
620           }
621         } else {
622           for (int replicaGetIndice : replicaGetIndices) {
623             addReplicaActions(replicaGetIndice, actionsByServer, unknownLocActions);
624           }
625         }
626         if (!actionsByServer.isEmpty()) {
627           sendMultiAction(actionsByServer, 1, null, unknownLocActions.isEmpty());
628         }
629         if (!unknownLocActions.isEmpty()) {
630           actionsByServer = new HashMap<ServerName, MultiAction<Row>>();
631           for (Action<Row> action : unknownLocActions) {
632             addReplicaActionsAgain(action, actionsByServer);
633           }
634           // Some actions may have completely failed, they are handled inside addAgain.
635           if (!actionsByServer.isEmpty()) {
636             sendMultiAction(actionsByServer, 1, null, true);
637           }
638         }
639       }
640 
641       /**
642        * Add replica actions to action map by server.
643        * @param index Index of the original action.
644        * @param actionsByServer The map by server to add it to.
645        */
646       private void addReplicaActions(int index, Map<ServerName, MultiAction<Row>> actionsByServer,
647           List<Action<Row>> unknownReplicaActions) {
648         if (results[index] != null) return; // opportunistic. Never goes from non-null to null.
649         Action<Row> action = initialActions.get(index);
650         RegionLocations loc = findAllLocationsOrFail(action, true);
651         if (loc == null) return;
652         HRegionLocation[] locs = loc.getRegionLocations();
653         if (locs.length == 1) {
654           LOG.warn("No replicas found for " + action.getAction());
655           return;
656         }
657         synchronized (replicaResultLock) {
658           // Don't run replica calls if the original has finished. We could do it e.g. if
659           // original has already failed before first replica call (unlikely given retries),
660           // but that would require additional synchronization w.r.t. returning to caller.
661           if (results[index] != null) return;
662           // We set the number of calls here. After that any path must call setResult/setError.
663           // True even for replicas that are not found - if we refuse to send we MUST set error.
664           results[index] = new ReplicaResultState(locs.length);
665         }
666         for (int i = 1; i < locs.length; ++i) {
667           Action<Row> replicaAction = new Action<Row>(action, i);
668           if (locs[i] != null) {
669             addAction(locs[i].getServerName(), locs[i].getRegionInfo().getRegionName(),
670                 replicaAction, actionsByServer, nonceGroup);
671           } else {
672             unknownReplicaActions.add(replicaAction);
673           }
674         }
675       }
676 
677       private void addReplicaActionsAgain(
678           Action<Row> action, Map<ServerName, MultiAction<Row>> actionsByServer) {
679         if (action.getReplicaId() == RegionReplicaUtil.DEFAULT_REPLICA_ID) {
680           throw new AssertionError("Cannot have default replica here");
681         }
682         HRegionLocation loc = getReplicaLocationOrFail(action);
683         if (loc == null) return;
684         addAction(loc.getServerName(), loc.getRegionInfo().getRegionName(),
685             action, actionsByServer, nonceGroup);
686       }
687     }
688 
689     /**
690      * Runnable (that can be submitted to thread pool) that submits MultiAction to a
691      * single server. The server call is synchronous, therefore we do it on a thread pool.
692      */
693     private final class SingleServerRequestRunnable implements Runnable {
694       private final MultiAction<Row> multiAction;
695       private final int numAttempt;
696       private final ServerName server;
697       private final Set<MultiServerCallable<Row>> callsInProgress;
698 
699       private SingleServerRequestRunnable(
700           MultiAction<Row> multiAction, int numAttempt, ServerName server,
701           Set<MultiServerCallable<Row>> callsInProgress) {
702         this.multiAction = multiAction;
703         this.numAttempt = numAttempt;
704         this.server = server;
705         this.callsInProgress = callsInProgress;
706       }
707 
708       @Override
709       public void run() {
710         MultiResponse res;
711         MultiServerCallable<Row> callable = null;
712         try {
713           callable = createCallable(server, tableName, multiAction);
714           try {
715             RpcRetryingCaller<MultiResponse> caller = createCaller(callable);
716             if (callsInProgress != null) callsInProgress.add(callable);
717             res = caller.callWithoutRetries(callable, timeout);
718 
719             if (res == null) {
720               // Cancelled
721               return;
722             }
723 
724           } catch (IOException e) {
725             // The service itself failed . It may be an error coming from the communication
726             //   layer, but, as well, a functional error raised by the server.
727             receiveGlobalFailure(multiAction, server, numAttempt, e);
728             return;
729           } catch (Throwable t) {
730             // This should not happen. Let's log & retry anyway.
731             LOG.error("#" + id + ", Caught throwable while calling. This is unexpected." +
732                 " Retrying. Server is " + server + ", tableName=" + tableName, t);
733             receiveGlobalFailure(multiAction, server, numAttempt, t);
734             return;
735           }
736 
737           // Normal case: we received an answer from the server, and it's not an exception.
738           receiveMultiAction(multiAction, server, res, numAttempt);
739         } catch (Throwable t) {
740               // Something really bad happened. We are on the send thread that will now die.
741               LOG.error("Internal AsyncProcess #" + id + " error for "
742                   + tableName + " processing for " + server, t);
743               throw new RuntimeException(t);
744         } finally {
745           decTaskCounters(multiAction.getRegions(), server);
746           if (callsInProgress != null && callable != null) {
747             callsInProgress.remove(callable);
748           }
749         }
750       }
751     }
752 
753     private final Batch.Callback<CResult> callback;
754     private final BatchErrors errors;
755     private final ConnectionManager.ServerErrorTracker errorsByServer;
756     private final ExecutorService pool;
757     private final Set<MultiServerCallable<Row>> callsInProgress;
758 
759 
760     private final TableName tableName;
761     private final AtomicLong actionsInProgress = new AtomicLong(-1);
762     /** The lock controls access to results. It is only held when populating results where
763      * there might be several callers (eventual consistency gets). For other requests,
764      * there's one unique call going on per result index. */
765     private final Object replicaResultLock = new Object();
766     /** Result array.  Null if results are not needed. Otherwise, each index corresponds to
767      * the action index in initial actions submitted. For most request types, has null-s for
768      * requests that are not done, and result/exception for those that are done.
769      * For eventual-consistency gets, initially the same applies; at some point, replica calls
770      * might be started, and ReplicaResultState is put at the corresponding indices. The
771      * returning calls check the type to detect when this is the case. After all calls are done,
772      * ReplicaResultState-s are replaced with results for the user. */
773     private final Object[] results;
774     /** Indices of replica gets in results. If null, all or no actions are replica-gets. */
775     private final int[] replicaGetIndices;
776     private final boolean hasAnyReplicaGets;
777     private final long nonceGroup;
778 
779     public AsyncRequestFutureImpl(TableName tableName, List<Action<Row>> actions, long nonceGroup,
780         ExecutorService pool, boolean needResults, Object[] results,
781         Batch.Callback<CResult> callback) {
782       this.pool = pool;
783       this.callback = callback;
784       this.nonceGroup = nonceGroup;
785       this.tableName = tableName;
786       this.actionsInProgress.set(actions.size());
787       if (results != null) {
788         assert needResults;
789         if (results.length != actions.size()) throw new AssertionError("results.length");
790         this.results = results;
791         for (int i = 0; i != this.results.length; ++i) {
792           results[i] = null;
793         }
794       } else {
795         this.results = needResults ? new Object[actions.size()] : null;
796       }
797       List<Integer> replicaGetIndices = null;
798       boolean hasAnyReplicaGets = false;
799       if (needResults) {
800         // Check to see if any requests might require replica calls.
801         // We expect that many requests will consist of all or no multi-replica gets; in such
802         // cases we would just use a boolean (hasAnyReplicaGets). If there's a mix, we will
803         // store the list of action indexes for which replica gets are possible, and set
804         // hasAnyReplicaGets to true.
805         boolean hasAnyNonReplicaReqs = false;
806         int posInList = 0;
807         for (Action<Row> action : actions) {
808           boolean isReplicaGet = isReplicaGet(action.getAction());
809           if (isReplicaGet) {
810             hasAnyReplicaGets = true;
811             if (hasAnyNonReplicaReqs) { // Mixed case
812               if (replicaGetIndices == null) {
813                 replicaGetIndices = new ArrayList<Integer>(actions.size() - 1);
814               }
815               replicaGetIndices.add(posInList);
816             }
817           } else if (!hasAnyNonReplicaReqs) {
818             // The first non-multi-replica request in the action list.
819             hasAnyNonReplicaReqs = true;
820             if (posInList > 0) {
821               // Add all the previous requests to the index lists. We know they are all
822               // replica-gets because this is the first non-multi-replica request in the list.
823               replicaGetIndices = new ArrayList<Integer>(actions.size() - 1);
824               for (int i = 0; i < posInList; ++i) {
825                 replicaGetIndices.add(i);
826               }
827             }
828           }
829           ++posInList;
830         }
831       }
832       this.hasAnyReplicaGets = hasAnyReplicaGets;
833       if (replicaGetIndices != null) {
834         this.replicaGetIndices = new int[replicaGetIndices.size()];
835         int i = 0;
836         for (Integer el : replicaGetIndices) {
837           this.replicaGetIndices[i++] = el;
838         }
839       } else {
840         this.replicaGetIndices = null;
841       }
842       this.callsInProgress = !hasAnyReplicaGets ? null :
843           Collections.newSetFromMap(new ConcurrentHashMap<MultiServerCallable<Row>, Boolean>());
844 
845       this.errorsByServer = createServerErrorTracker();
846       this.errors = (globalErrors != null) ? globalErrors : new BatchErrors();
847     }
848 
849     public Set<MultiServerCallable<Row>> getCallsInProgress() {
850       return callsInProgress;
851     }
852 
853     /**
854      * Group a list of actions per region servers, and send them.
855      *
856      * @param currentActions - the list of row to submit
857      * @param numAttempt - the current numAttempt (first attempt is 1)
858      */
859     private void groupAndSendMultiAction(List<Action<Row>> currentActions, int numAttempt) {
860       Map<ServerName, MultiAction<Row>> actionsByServer =
861           new HashMap<ServerName, MultiAction<Row>>();
862 
863       boolean isReplica = false;
864       List<Action<Row>> unknownReplicaActions = null;
865       for (Action<Row> action : currentActions) {
866         RegionLocations locs = findAllLocationsOrFail(action, true);
867         if (locs == null) continue;
868         boolean isReplicaAction = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
869         if (isReplica && !isReplicaAction) {
870           // This is the property of the current implementation, not a requirement.
871           throw new AssertionError("Replica and non-replica actions in the same retry");
872         }
873         isReplica = isReplicaAction;
874         HRegionLocation loc = locs.getRegionLocation(action.getReplicaId());
875         if (loc == null || loc.getServerName() == null) {
876           if (isReplica) {
877             if (unknownReplicaActions == null) {
878               unknownReplicaActions = new ArrayList<Action<Row>>();
879             }
880             unknownReplicaActions.add(action);
881           } else {
882             if (LOG.isInfoEnabled()) {
883               LOG.info("Failed to find location: " + loc + " for replica: " + action.getReplicaId() + " and action: " + action.getAction());
884             }
885             // TODO: relies on primary location always being fetched
886             manageLocationError(action, null);
887           }
888         } else {
889           byte[] regionName = loc.getRegionInfo().getRegionName();
890           addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
891         }
892       }
893       boolean doStartReplica = (numAttempt == 1 && !isReplica && hasAnyReplicaGets);
894       boolean hasUnknown = unknownReplicaActions != null && !unknownReplicaActions.isEmpty();
895 
896       if (!actionsByServer.isEmpty()) {
897         // If this is a first attempt to group and send, no replicas, we need replica thread.
898         sendMultiAction(actionsByServer, numAttempt, (doStartReplica && !hasUnknown)
899             ? currentActions : null, numAttempt > 1 && !hasUnknown);
900       }
901 
902       if (hasUnknown) {
903         actionsByServer = new HashMap<ServerName, MultiAction<Row>>();
904         for (Action<Row> action : unknownReplicaActions) {
905           HRegionLocation loc = getReplicaLocationOrFail(action);
906           if (loc == null) continue;
907           byte[] regionName = loc.getRegionInfo().getRegionName();
908           addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
909         }
910         if (!actionsByServer.isEmpty()) {
911           sendMultiAction(
912               actionsByServer, numAttempt, doStartReplica ? currentActions : null, true);
913         }
914       }
915     }
916 
917     private HRegionLocation getReplicaLocationOrFail(Action<Row> action) {
918       // We are going to try get location once again. For each action, we'll do it once
919       // from cache, because the previous calls in the loop might populate it.
920       int replicaId = action.getReplicaId();
921       RegionLocations locs = findAllLocationsOrFail(action, true);
922       if (locs == null) return null; // manageError already called
923       HRegionLocation loc = locs.getRegionLocation(replicaId);
924       if (loc == null || loc.getServerName() == null) {
925         locs = findAllLocationsOrFail(action, false);
926         if (locs == null) return null; // manageError already called
927         loc = locs.getRegionLocation(replicaId);
928       }
929       if (loc == null || loc.getServerName() == null) {
930         if (LOG.isInfoEnabled()) {
931           LOG.info("Failed to find location: " + loc + " for replica: " + replicaId + " and action: " + action.getAction());
932         }
933         manageLocationError(action, null);
934         return null;
935       }
936       return loc;
937     }
938 
939     private void manageLocationError(Action<Row> action, Exception ex) {
940       String msg = "Cannot get replica " + action.getReplicaId()
941           + " location for " + action.getAction();
942       if (ex == null) {
943         LOG.error(msg);
944         ex = new IOException(msg);
945       } else {
946         LOG.error(msg, ex);
947       }
948       LOG.error(msg, ex);
949       manageError(action.getOriginalIndex(), action.getAction(),
950           Retry.NO_LOCATION_PROBLEM, ex, null);
951     }
952 
953     private RegionLocations findAllLocationsOrFail(Action<Row> action, boolean useCache) {
954       if (action.getAction() == null) throw new IllegalArgumentException("#" + id +
955           ", row cannot be null");
956       RegionLocations loc = null;
957       try {
958         loc = connection.locateRegion(
959             tableName, action.getAction().getRow(), useCache, true, action.getReplicaId());
960       } catch (IOException ex) {
961         manageLocationError(action, ex);
962       }
963       return loc;
964     }
965 
966     /**
967      * Send a multi action structure to the servers, after a delay depending on the attempt
968      * number. Asynchronous.
969      *
970      * @param actionsByServer the actions structured by regions
971      * @param numAttempt the attempt number.
972      * @param actionsForReplicaThread original actions for replica thread; null on non-first call.
973      */
974     private void sendMultiAction(Map<ServerName, MultiAction<Row>> actionsByServer,
975         int numAttempt, List<Action<Row>> actionsForReplicaThread, boolean reuseThread) {
976       // Run the last item on the same thread if we are already on a send thread.
977       // We hope most of the time it will be the only item, so we can cut down on threads.
978       int actionsRemaining = actionsByServer.size();
979       // This iteration is by server (the HRegionLocation comparator is by server portion only).
980       for (Map.Entry<ServerName, MultiAction<Row>> e : actionsByServer.entrySet()) {
981         ServerName server = e.getKey();
982         MultiAction<Row> multiAction = e.getValue();
983         incTaskCounters(multiAction.getRegions(), server);
984         Collection<? extends Runnable> runnables = getNewMultiActionRunnable(server, multiAction,
985             numAttempt);
986         // make sure we correctly count the number of runnables before we try to reuse the send
987         // thread, in case we had to split the request into different runnables because of backoff
988         if (runnables.size() > actionsRemaining) {
989           actionsRemaining = runnables.size();
990         }
991 
992         // run all the runnables
993         for (Runnable runnable : runnables) {
994           if ((--actionsRemaining == 0) && reuseThread) {
995             runnable.run();
996           } else {
997             try {
998               pool.submit(runnable);
999             } catch (Throwable t) {
1000               if (t instanceof RejectedExecutionException) {
1001                 // This should never happen. But as the pool is provided by the end user,
1002                // let's secure this a little.
1003                LOG.warn("#" + id + ", the task was rejected by the pool. This is unexpected." +
1004                   " Server is " + server.getServerName(), t);
1005               } else {
1006                 // see #HBASE-14359 for more details
1007                 LOG.warn("Caught unexpected exception/error: ", t);
1008               }
1009               decTaskCounters(multiAction.getRegions(), server);
1010               // We're likely to fail again, but this will increment the attempt counter,
1011              // so it will finish.
1012               receiveGlobalFailure(multiAction, server, numAttempt, t);
1013             }
1014           }
1015         }
1016       }
1017 
1018       if (actionsForReplicaThread != null) {
1019         startWaitingForReplicaCalls(actionsForReplicaThread);
1020       }
1021     }
1022 
1023     private Collection<? extends Runnable> getNewMultiActionRunnable(ServerName server,
1024         MultiAction<Row> multiAction,
1025         int numAttempt) {
1026       // no stats to manage, just do the standard action
1027       if (AsyncProcess.this.connection.getStatisticsTracker() == null) {
1028         return Collections.singletonList(Trace.wrap("AsyncProcess.sendMultiAction",
1029             new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress)));
1030       }
1031 
1032       // group the actions by the amount of delay
1033       Map<Long, DelayingRunner> actions = new HashMap<Long, DelayingRunner>(multiAction
1034           .size());
1035 
1036       // split up the actions
1037       for (Map.Entry<byte[], List<Action<Row>>> e : multiAction.actions.entrySet()) {
1038         Long backoff = getBackoff(server, e.getKey());
1039         DelayingRunner runner = actions.get(backoff);
1040         if (runner == null) {
1041           actions.put(backoff, new DelayingRunner(backoff, e));
1042         } else {
1043           runner.add(e);
1044         }
1045       }
1046 
1047       List<Runnable> toReturn = new ArrayList<Runnable>(actions.size());
1048       for (DelayingRunner runner : actions.values()) {
1049         String traceText = "AsyncProcess.sendMultiAction";
1050         Runnable runnable =
1051             new SingleServerRequestRunnable(runner.getActions(), numAttempt, server,
1052                 callsInProgress);
1053         // use a delay runner only if we need to sleep for some time
1054         if (runner.getSleepTime() > 0) {
1055           runner.setRunner(runnable);
1056           traceText = "AsyncProcess.clientBackoff.sendMultiAction";
1057           runnable = runner;
1058         }
1059         runnable = Trace.wrap(traceText, runnable);
1060         toReturn.add(runnable);
1061 
1062       }
1063       return toReturn;
1064     }
1065 
1066     /**
1067      * @param server server location where the target region is hosted
1068      * @param regionName name of the region which we are going to write some data
1069      * @return the amount of time the client should wait until it submit a request to the
1070      * specified server and region
1071      */
1072     private Long getBackoff(ServerName server, byte[] regionName) {
1073       ServerStatisticTracker tracker = AsyncProcess.this.connection.getStatisticsTracker();
1074       ServerStatistics stats = tracker.getStats(server);
1075       return AsyncProcess.this.connection.getBackoffPolicy()
1076           .getBackoffTime(server, regionName, stats);
1077     }
1078 
1079     /**
1080      * Starts waiting to issue replica calls on a different thread; or issues them immediately.
1081      */
1082     private void startWaitingForReplicaCalls(List<Action<Row>> actionsForReplicaThread) {
1083       long startTime = EnvironmentEdgeManager.currentTime();
1084       ReplicaCallIssuingRunnable replicaRunnable = new ReplicaCallIssuingRunnable(
1085           actionsForReplicaThread, startTime);
1086       if (primaryCallTimeoutMicroseconds == 0) {
1087         // Start replica calls immediately.
1088         replicaRunnable.run();
1089       } else {
1090         // Start the thread that may kick off replica gets.
1091         // TODO: we could do it on the same thread, but it's a user thread, might be a bad idea.
1092         try {
1093           pool.submit(replicaRunnable);
1094         } catch (RejectedExecutionException ree) {
1095           LOG.warn("#" + id + ", replica task was rejected by the pool - no replica calls", ree);
1096         }
1097       }
1098     }
1099 
1100     /**
1101      * Check that we can retry acts accordingly: logs, set the error status.
1102      *
1103      * @param originalIndex the position in the list sent
1104      * @param row           the row
1105      * @param canRetry      if false, we won't retry whatever the settings.
1106      * @param throwable     the throwable, if any (can be null)
1107      * @param server        the location, if any (can be null)
1108      * @return true if the action can be retried, false otherwise.
1109      */
1110     public Retry manageError(int originalIndex, Row row, Retry canRetry,
1111                                 Throwable throwable, ServerName server) {
1112       if (canRetry == Retry.YES
1113           && throwable != null && throwable instanceof DoNotRetryIOException) {
1114         canRetry = Retry.NO_NOT_RETRIABLE;
1115       }
1116 
1117       if (canRetry != Retry.YES) {
1118         // Batch.Callback<Res> was not called on failure in 0.94. We keep this.
1119         setError(originalIndex, row, throwable, server);
1120       } else if (isActionComplete(originalIndex, row)) {
1121         canRetry = Retry.NO_OTHER_SUCCEEDED;
1122       }
1123       return canRetry;
1124     }
1125 
1126     /**
1127      * Resubmit all the actions from this multiaction after a failure.
1128      *
1129      * @param rsActions  the actions still to do from the initial list
1130      * @param server   the destination
1131      * @param numAttempt the number of attempts so far
1132      * @param t the throwable (if any) that caused the resubmit
1133      */
1134     private void receiveGlobalFailure(
1135         MultiAction<Row> rsActions, ServerName server, int numAttempt, Throwable t) {
1136       errorsByServer.reportServerError(server);
1137       Retry canRetry = errorsByServer.canRetryMore(numAttempt)
1138           ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED;
1139 
1140       if (tableName == null) {
1141         // tableName is null when we made a cross-table RPC call.
1142         connection.clearCaches(server);
1143       }
1144       int failed = 0, stopped = 0;
1145       List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
1146       for (Map.Entry<byte[], List<Action<Row>>> e : rsActions.actions.entrySet()) {
1147         byte[] regionName = e.getKey();
1148         byte[] row = e.getValue().iterator().next().getAction().getRow();
1149         // Do not use the exception for updating cache because it might be coming from
1150         // any of the regions in the MultiAction.
1151         // TODO: depending on type of exception we might not want to update cache at all?
1152         if (tableName != null) {
1153           connection.updateCachedLocations(tableName, regionName, row, null, server);
1154         }
1155         for (Action<Row> action : e.getValue()) {
1156           Retry retry = manageError(
1157               action.getOriginalIndex(), action.getAction(), canRetry, t, server);
1158           if (retry == Retry.YES) {
1159             toReplay.add(action);
1160           } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
1161             ++stopped;
1162           } else {
1163             ++failed;
1164           }
1165         }
1166       }
1167 
1168       if (toReplay.isEmpty()) {
1169         logNoResubmit(server, numAttempt, rsActions.size(), t, failed, stopped);
1170       } else {
1171         resubmit(server, toReplay, numAttempt, rsActions.size(), t);
1172       }
1173     }
1174 
1175     /**
1176      * Log as much info as possible, and, if there is something to replay,
1177      * submit it again after a back off sleep.
1178      */
1179     private void resubmit(ServerName oldServer, List<Action<Row>> toReplay,
1180         int numAttempt, int failureCount, Throwable throwable) {
1181       // We have something to replay. We're going to sleep a little before.
1182 
1183       // We have two contradicting needs here:
1184       //  1) We want to get the new location after having slept, as it may change.
1185       //  2) We want to take into account the location when calculating the sleep time.
1186       // It should be possible to have some heuristics to take the right decision. Short term,
1187       //  we go for one.
1188       long backOffTime = errorsByServer.calculateBackoffTime(oldServer, pause);
1189       if (numAttempt > startLogErrorsCnt) {
1190         // We use this value to have some logs when we have multiple failures, but not too many
1191         //  logs, as errors are to be expected when a region moves, splits and so on
1192         LOG.info(createLog(numAttempt, failureCount, toReplay.size(),
1193             oldServer, throwable, backOffTime, true, null, -1, -1));
1194       }
1195 
1196       try {
1197         Thread.sleep(backOffTime);
1198       } catch (InterruptedException e) {
1199         LOG.warn("#" + id + ", not sent: " + toReplay.size() + " operations, " + oldServer, e);
1200         Thread.currentThread().interrupt();
1201         return;
1202       }
1203 
1204       groupAndSendMultiAction(toReplay, numAttempt + 1);
1205     }
1206 
1207     private void logNoResubmit(ServerName oldServer, int numAttempt,
1208         int failureCount, Throwable throwable, int failed, int stopped) {
1209       if (failureCount != 0 || numAttempt > startLogErrorsCnt + 1) {
1210         String timeStr = new Date(errorsByServer.getStartTrackingTime()).toString();
1211         String logMessage = createLog(numAttempt, failureCount, 0, oldServer,
1212             throwable, -1, false, timeStr, failed, stopped);
1213         if (failed != 0) {
1214           // Only log final failures as warning
1215           LOG.warn(logMessage);
1216         } else {
1217           LOG.info(logMessage);
1218         }
1219       }
1220     }
1221 
1222     /**
1223      * Called when we receive the result of a server query.
1224      *
1225      * @param multiAction    - the multiAction we sent
1226      * @param server       - the location. It's used as a server name.
1227      * @param responses      - the response, if any
1228      * @param numAttempt     - the attempt
1229      */
1230     private void receiveMultiAction(MultiAction<Row> multiAction,
1231         ServerName server, MultiResponse responses, int numAttempt) {
1232        assert responses != null;
1233 
1234       // Success or partial success
1235       // Analyze detailed results. We can still have individual failures to be redo.
1236       // two specific throwables are managed:
1237       //  - DoNotRetryIOException: we continue to retry for other actions
1238       //  - RegionMovedException: we update the cache with the new region location
1239 
1240       List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
1241       Throwable throwable = null;
1242       int failureCount = 0;
1243       boolean canRetry = true;
1244 
1245       // Go by original action.
1246       int failed = 0, stopped = 0;
1247       for (Map.Entry<byte[], List<Action<Row>>> regionEntry : multiAction.actions.entrySet()) {
1248         byte[] regionName = regionEntry.getKey();
1249         Map<Integer, Object> regionResults = responses.getResults().get(regionName);
1250         if (regionResults == null) {
1251           if (!responses.getExceptions().containsKey(regionName)) {
1252             LOG.error("Server sent us neither results nor exceptions for "
1253                 + Bytes.toStringBinary(regionName));
1254             responses.getExceptions().put(regionName, new RuntimeException("Invalid response"));
1255           }
1256           continue;
1257         }
1258         boolean regionFailureRegistered = false;
1259         for (Action<Row> sentAction : regionEntry.getValue()) {
1260           Object result = regionResults.get(sentAction.getOriginalIndex());
1261           // Failure: retry if it's make sense else update the errors lists
1262           if (result == null || result instanceof Throwable) {
1263             Row row = sentAction.getAction();
1264             // Register corresponding failures once per server/once per region.
1265             if (!regionFailureRegistered) {
1266               regionFailureRegistered = true;
1267               connection.updateCachedLocations(
1268                   tableName, regionName, row.getRow(), result, server);
1269             }
1270             if (failureCount == 0) {
1271               errorsByServer.reportServerError(server);
1272               // We determine canRetry only once for all calls, after reporting server failure.
1273               canRetry = errorsByServer.canRetryMore(numAttempt);
1274             }
1275             ++failureCount;
1276             Retry retry = manageError(sentAction.getOriginalIndex(), row,
1277                 canRetry ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED, (Throwable)result, server);
1278             if (retry == Retry.YES) {
1279               toReplay.add(sentAction);
1280             } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
1281               ++stopped;
1282             } else {
1283               ++failed;
1284             }
1285           } else {
1286             // update the stats about the region, if its a user table. We don't want to slow down
1287             // updates to meta tables, especially from internal updates (master, etc).
1288             if (AsyncProcess.this.connection.getStatisticsTracker() != null) {
1289               result = ResultStatsUtil.updateStats(result,
1290                   AsyncProcess.this.connection.getStatisticsTracker(), server, regionName);
1291             }
1292 
1293             if (callback != null) {
1294               try {
1295                 //noinspection unchecked
1296                 // TODO: would callback expect a replica region name if it gets one?
1297                 this.callback.update(regionName, sentAction.getAction().getRow(), (CResult)result);
1298               } catch (Throwable t) {
1299                 LOG.error("User callback threw an exception for "
1300                     + Bytes.toStringBinary(regionName) + ", ignoring", t);
1301               }
1302             }
1303             setResult(sentAction, result);
1304           }
1305         }
1306       }
1307 
1308       // The failures global to a region. We will use for multiAction we sent previously to find the
1309       //   actions to replay.
1310       for (Map.Entry<byte[], Throwable> throwableEntry : responses.getExceptions().entrySet()) {
1311         throwable = throwableEntry.getValue();
1312         byte[] region = throwableEntry.getKey();
1313         List<Action<Row>> actions = multiAction.actions.get(region);
1314         if (actions == null || actions.isEmpty()) {
1315           throw new IllegalStateException("Wrong response for the region: " +
1316               HRegionInfo.encodeRegionName(region));
1317         }
1318 
1319         if (failureCount == 0) {
1320           errorsByServer.reportServerError(server);
1321           canRetry = errorsByServer.canRetryMore(numAttempt);
1322         }
1323         if (null == tableName) {
1324           // For multi-actions, we don't have a table name, but we want to make sure to clear the
1325           // cache in case there were location-related exceptions
1326           connection.clearCaches(server);
1327         } else {
1328           connection.updateCachedLocations(
1329               tableName, region, actions.get(0).getAction().getRow(), throwable, server);
1330         }
1331         failureCount += actions.size();
1332 
1333         for (Action<Row> action : actions) {
1334           Row row = action.getAction();
1335           Retry retry = manageError(action.getOriginalIndex(), row,
1336               canRetry ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED, throwable, server);
1337           if (retry == Retry.YES) {
1338             toReplay.add(action);
1339           } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
1340             ++stopped;
1341           } else {
1342             ++failed;
1343           }
1344         }
1345       }
1346 
1347       if (toReplay.isEmpty()) {
1348         logNoResubmit(server, numAttempt, failureCount, throwable, failed, stopped);
1349       } else {
1350         resubmit(server, toReplay, numAttempt, failureCount, throwable);
1351       }
1352     }
1353 
1354     private String createLog(int numAttempt, int failureCount, int replaySize, ServerName sn,
1355         Throwable error, long backOffTime, boolean willRetry, String startTime,
1356         int failed, int stopped) {
1357       StringBuilder sb = new StringBuilder();
1358       sb.append("#").append(id).append(", table=").append(tableName).append(", ")
1359         .append("attempt=").append(numAttempt)
1360         .append("/").append(numTries).append(" ");
1361 
1362       if (failureCount > 0 || error != null){
1363         sb.append("failed=").append(failureCount).append("ops").append(", last exception: ").
1364             append(error == null ? "null" : error);
1365       } else {
1366         sb.append("succeeded");
1367       }
1368 
1369       sb.append(" on ").append(sn).append(", tracking started ").append(startTime);
1370 
1371       if (willRetry) {
1372         sb.append(", retrying after=").append(backOffTime).append("ms").
1373             append(", replay=").append(replaySize).append("ops");
1374       } else if (failureCount > 0) {
1375         if (stopped > 0) {
1376           sb.append("; not retrying ").append(stopped).append(" due to success from other replica");
1377         }
1378         if (failed > 0) {
1379           sb.append("; not retrying ").append(failed).append(" - final failure");
1380         }
1381 
1382       }
1383 
1384       return sb.toString();
1385     }
1386 
1387     /**
1388      * Sets the non-error result from a particular action.
1389      * @param action Action (request) that the server responded to.
1390      * @param result The result.
1391      */
1392     private void setResult(Action<Row> action, Object result) {
1393       if (result == null) {
1394         throw new RuntimeException("Result cannot be null");
1395       }
1396       ReplicaResultState state = null;
1397       boolean isStale = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
1398       int index = action.getOriginalIndex();
1399       if (results == null) {
1400          decActionCounter(index);
1401          return; // Simple case, no replica requests.
1402       } else if ((state = trySetResultSimple(
1403           index, action.getAction(), false, result, null, isStale)) == null) {
1404         return; // Simple case, no replica requests.
1405       }
1406       assert state != null;
1407       // At this point we know that state is set to replica tracking class.
1408       // It could be that someone else is also looking at it; however, we know there can
1409       // only be one state object, and only one thread can set callCount to 0. Other threads
1410       // will either see state with callCount 0 after locking it; or will not see state at all
1411       // we will replace it with the result.
1412       synchronized (state) {
1413         if (state.callCount == 0) return; // someone already set the result
1414         state.callCount = 0;
1415       }
1416       synchronized (replicaResultLock) {
1417         if (results[index] != state) {
1418           throw new AssertionError("We set the callCount but someone else replaced the result");
1419         }
1420         results[index] = result;
1421       }
1422 
1423       decActionCounter(index);
1424     }
1425 
1426     /**
1427      * Sets the error from a particular action.
1428      * @param index Original action index.
1429      * @param row Original request.
1430      * @param throwable The resulting error.
1431      * @param server The source server.
1432      */
1433     private void setError(int index, Row row, Throwable throwable, ServerName server) {
1434       ReplicaResultState state = null;
1435       if (results == null) {
1436         // Note that we currently cannot have replica requests with null results. So it shouldn't
1437         // happen that multiple replica calls will call dAC for same actions with results == null.
1438         // Only one call per action should be present in this case.
1439         errors.add(throwable, row, server);
1440         decActionCounter(index);
1441         return; // Simple case, no replica requests.
1442       } else if ((state = trySetResultSimple(
1443           index, row, true, throwable, server, false)) == null) {
1444         return; // Simple case, no replica requests.
1445       }
1446       assert state != null;
1447       BatchErrors target = null; // Error will be added to final errors, or temp replica errors.
1448       boolean isActionDone = false;
1449       synchronized (state) {
1450         switch (state.callCount) {
1451           case 0: return; // someone already set the result
1452           case 1: { // All calls failed, we are the last error.
1453             target = errors;
1454             isActionDone = true;
1455             break;
1456           }
1457           default: {
1458             assert state.callCount > 1;
1459             if (state.replicaErrors == null) {
1460               state.replicaErrors = new BatchErrors();
1461             }
1462             target = state.replicaErrors;
1463             break;
1464           }
1465         }
1466         --state.callCount;
1467       }
1468       target.add(throwable, row, server);
1469       if (isActionDone) {
1470         if (state.replicaErrors != null) { // last call, no need to lock
1471           errors.merge(state.replicaErrors);
1472         }
1473         // See setResult for explanations.
1474         synchronized (replicaResultLock) {
1475           if (results[index] != state) {
1476             throw new AssertionError("We set the callCount but someone else replaced the result");
1477           }
1478           results[index] = throwable;
1479         }
1480         decActionCounter(index);
1481       }
1482     }
1483 
1484     /**
1485      * Checks if the action is complete; used on error to prevent needless retries.
1486      * Does not synchronize, assuming element index/field accesses are atomic.
1487      * This is an opportunistic optimization check, doesn't have to be strict.
1488      * @param index Original action index.
1489      * @param row Original request.
1490      */
1491     private boolean isActionComplete(int index, Row row) {
1492       if (!isReplicaGet(row)) return false;
1493       Object resObj = results[index];
1494       return (resObj != null) && (!(resObj instanceof ReplicaResultState)
1495           || ((ReplicaResultState)resObj).callCount == 0);
1496     }
1497 
1498     /**
1499      * Tries to set the result or error for a particular action as if there were no replica calls.
1500      * @return null if successful; replica state if there were in fact replica calls.
1501      */
1502     private ReplicaResultState trySetResultSimple(int index, Row row, boolean isError,
1503         Object result, ServerName server, boolean isFromReplica) {
1504       Object resObj = null;
1505       if (!isReplicaGet(row)) {
1506         if (isFromReplica) {
1507           throw new AssertionError("Unexpected stale result for " + row);
1508         }
1509         results[index] = result;
1510       } else {
1511         synchronized (replicaResultLock) {
1512           if ((resObj = results[index]) == null) {
1513             if (isFromReplica) {
1514               throw new AssertionError("Unexpected stale result for " + row);
1515             }
1516             results[index] = result;
1517           }
1518         }
1519       }
1520 
1521       ReplicaResultState rrs =
1522           (resObj instanceof ReplicaResultState) ? (ReplicaResultState)resObj : null;
1523       if (rrs == null && isError) {
1524         // The resObj is not replica state (null or already set).
1525         errors.add((Throwable)result, row, server);
1526       }
1527 
1528       if (resObj == null) {
1529         // resObj is null - no replica calls were made.
1530         decActionCounter(index);
1531         return null;
1532       }
1533       return rrs;
1534     }
1535 
1536     private void decActionCounter(int index) {
1537       long actionsRemaining = actionsInProgress.decrementAndGet();
1538       if (actionsRemaining < 0) {
1539         String error = buildDetailedErrorMsg("Incorrect actions in progress", index);
1540         throw new AssertionError(error);
1541       } else if (actionsRemaining == 0) {
1542         synchronized (actionsInProgress) {
1543           actionsInProgress.notifyAll();
1544         }
1545       }
1546     }
1547 
1548     private String buildDetailedErrorMsg(String string, int index) {
1549       String error = string + "; called for " + index +
1550           ", actionsInProgress " + actionsInProgress.get() + "; replica gets: ";
1551       if (replicaGetIndices != null) {
1552         for (int i = 0; i < replicaGetIndices.length; ++i) {
1553           error += replicaGetIndices[i] + ", ";
1554         }
1555       } else {
1556         error += (hasAnyReplicaGets ? "all" : "none");
1557       }
1558       error += "; results ";
1559       if (results != null) {
1560         for (int i = 0; i < results.length; ++i) {
1561           Object o = results[i];
1562           error += ((o == null) ? "null" : o.toString()) + ", ";
1563         }
1564       }
1565       return error;
1566     }
1567 
1568     @Override
1569     public void waitUntilDone() throws InterruptedIOException {
1570       try {
1571         waitUntilDone(Long.MAX_VALUE);
1572       } catch (InterruptedException iex) {
1573         throw new InterruptedIOException(iex.getMessage());
1574       } finally {
1575         if (callsInProgress != null) {
1576           for (MultiServerCallable<Row> clb : callsInProgress) {
1577             clb.cancel();
1578           }
1579         }
1580       }
1581     }
1582 
1583     private boolean waitUntilDone(long cutoff) throws InterruptedException {
1584       boolean hasWait = cutoff != Long.MAX_VALUE;
1585       long lastLog = EnvironmentEdgeManager.currentTime();
1586       long currentInProgress;
1587       while (0 != (currentInProgress = actionsInProgress.get())) {
1588         long now = EnvironmentEdgeManager.currentTime();
1589         if (hasWait && (now * 1000L) > cutoff) {
1590           return false;
1591         }
1592         if (!hasWait) { // Only log if wait is infinite.
1593           if (now > lastLog + 10000) {
1594             lastLog = now;
1595             LOG.info("#" + id + ", waiting for " + currentInProgress + "  actions to finish");
1596           }
1597         }
1598         synchronized (actionsInProgress) {
1599           if (actionsInProgress.get() == 0) break;
1600           if (!hasWait) {
1601             actionsInProgress.wait(10);
1602           } else {
1603             long waitMicroSecond = Math.min(100000L, (cutoff - now * 1000L));
1604             TimeUnit.MICROSECONDS.timedWait(actionsInProgress, waitMicroSecond);
1605           }
1606         }
1607       }
1608       return true;
1609     }
1610 
1611     @Override
1612     public boolean hasError() {
1613       return errors.hasErrors();
1614     }
1615 
1616     @Override
1617     public List<? extends Row> getFailedOperations() {
1618       return errors.actions;
1619     }
1620 
1621     @Override
1622     public RetriesExhaustedWithDetailsException getErrors() {
1623       return errors.makeException();
1624     }
1625 
1626     @Override
1627     public Object[] getResults() throws InterruptedIOException {
1628       waitUntilDone();
1629       return results;
1630     }
1631   }
1632 
1633   @VisibleForTesting
1634   /** Create AsyncRequestFuture. Isolated to be easily overridden in the tests. */
1635   protected <CResult> AsyncRequestFutureImpl<CResult> createAsyncRequestFuture(
1636       TableName tableName, List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
1637       Batch.Callback<CResult> callback, Object[] results, boolean needResults) {
1638     return new AsyncRequestFutureImpl<CResult>(
1639         tableName, actions, nonceGroup, getPool(pool), needResults, results, callback);
1640   }
1641 
1642   /**
1643    * Create a callable. Isolated to be easily overridden in the tests.
1644    */
1645   @VisibleForTesting
1646   protected MultiServerCallable<Row> createCallable(final ServerName server,
1647       TableName tableName, final MultiAction<Row> multi) {
1648     return new MultiServerCallable<Row>(connection, tableName, server, this.rpcFactory, multi);
1649   }
1650 
1651   /**
1652    * Create a caller. Isolated to be easily overridden in the tests.
1653    */
1654   @VisibleForTesting
1655   protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
1656     return rpcCallerFactory.<MultiResponse> newCaller();
1657   }
1658 
1659   @VisibleForTesting
1660   /** Waits until all outstanding tasks are done. Used in tests. */
1661   void waitUntilDone() throws InterruptedIOException {
1662     waitForMaximumCurrentTasks(0, null);
1663   }
1664 
1665   /** Wait until the async does not have more than max tasks in progress. */
1666   private void waitForMaximumCurrentTasks(int max, String tableName)
1667       throws InterruptedIOException {
1668     waitForMaximumCurrentTasks(max, tasksInProgress, id, tableName);
1669   }
1670 
1671   // Break out this method so testable
1672   @VisibleForTesting
1673   void waitForMaximumCurrentTasks(int max, final AtomicLong tasksInProgress, final long id,
1674       String tableName) throws InterruptedIOException {
1675     long lastLog = EnvironmentEdgeManager.currentTime();
1676     long currentInProgress, oldInProgress = Long.MAX_VALUE;
1677     while ((currentInProgress = tasksInProgress.get()) > max) {
1678       if (oldInProgress != currentInProgress) { // Wait for in progress to change.
1679         long now = EnvironmentEdgeManager.currentTime();
1680         if (now > lastLog + 10000) {
1681           lastLog = now;
1682           LOG.info("#" + id + ", waiting for some tasks to finish. Expected max="
1683               + max + ", tasksInProgress=" + currentInProgress +
1684               " hasError=" + hasError() + tableName == null ? "" : ", tableName=" + tableName);
1685           if (currentInProgress <= thresholdToLogUndoneTaskDetails) {
1686             logDetailsOfUndoneTasks(currentInProgress);
1687           }
1688         }
1689       }
1690       oldInProgress = currentInProgress;
1691       try {
1692         synchronized (tasksInProgress) {
1693           if (tasksInProgress.get() == oldInProgress) {
1694             tasksInProgress.wait(10);
1695           }
1696         }
1697       } catch (InterruptedException e) {
1698         throw new InterruptedIOException("#" + id + ", interrupted." +
1699             " currentNumberOfTask=" + currentInProgress);
1700       }
1701     }
1702   }
1703 
1704   private void logDetailsOfUndoneTasks(long taskInProgress) {
1705     ArrayList<ServerName> servers = new ArrayList<ServerName>();
1706     for (Map.Entry<ServerName, AtomicInteger> entry : taskCounterPerServer.entrySet()) {
1707       if (entry.getValue().get() > 0) {
1708         servers.add(entry.getKey());
1709       }
1710     }
1711     LOG.info("Left over " + taskInProgress + " task(s) are processed on server(s): " + servers);
1712     if (taskInProgress <= THRESHOLD_TO_LOG_REGION_DETAILS) {
1713       ArrayList<String> regions = new ArrayList<String>();
1714       for (Map.Entry<byte[], AtomicInteger> entry : taskCounterPerRegion.entrySet()) {
1715         if (entry.getValue().get() > 0) {
1716           regions.add(Bytes.toString(entry.getKey()));
1717         }
1718       }
1719       LOG.info("Regions against which left over task(s) are processed: " + regions);
1720     }
1721   }
1722 
1723   /**
1724    * Only used w/useGlobalErrors ctor argument, for HTable backward compat.
1725    * @return Whether there were any errors in any request since the last time
1726    *          {@link #waitForAllPreviousOpsAndReset(List)} was called, or AP was created.
1727    */
1728   public boolean hasError() {
1729     return globalErrors.hasErrors();
1730   }
1731 
1732   /**
1733    * Only used w/useGlobalErrors ctor argument, for HTable backward compat.
1734    * Waits for all previous operations to finish, and returns errors and (optionally)
1735    * failed operations themselves.
1736    * @param failedRows an optional list into which the rows that failed since the last time
1737    *        {@link #waitForAllPreviousOpsAndReset(List)} was called, or AP was created, are saved.
1738    * @param tableName name of the table
1739    * @return all the errors since the last time {@link #waitForAllPreviousOpsAndReset(List)}
1740    *          was called, or AP was created.
1741    */
1742   public RetriesExhaustedWithDetailsException waitForAllPreviousOpsAndReset(
1743       List<Row> failedRows, String tableName) throws InterruptedIOException {
1744     waitForMaximumCurrentTasks(0, tableName);
1745     if (!globalErrors.hasErrors()) {
1746       return null;
1747     }
1748     if (failedRows != null) {
1749       failedRows.addAll(globalErrors.actions);
1750     }
1751     RetriesExhaustedWithDetailsException result = globalErrors.makeException();
1752     globalErrors.clear();
1753     return result;
1754   }
1755 
1756   /**
1757    * increment the tasks counters for a given set of regions. MT safe.
1758    */
1759   protected void incTaskCounters(Collection<byte[]> regions, ServerName sn) {
1760     tasksInProgress.incrementAndGet();
1761 
1762     AtomicInteger serverCnt = taskCounterPerServer.get(sn);
1763     if (serverCnt == null) {
1764       taskCounterPerServer.putIfAbsent(sn, new AtomicInteger());
1765       serverCnt = taskCounterPerServer.get(sn);
1766     }
1767     serverCnt.incrementAndGet();
1768 
1769     for (byte[] regBytes : regions) {
1770       AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
1771       if (regionCnt == null) {
1772         regionCnt = new AtomicInteger();
1773         AtomicInteger oldCnt = taskCounterPerRegion.putIfAbsent(regBytes, regionCnt);
1774         if (oldCnt != null) {
1775           regionCnt = oldCnt;
1776         }
1777       }
1778       regionCnt.incrementAndGet();
1779     }
1780   }
1781 
1782   /**
1783    * Decrements the counters for a given region and the region server. MT Safe.
1784    */
1785   protected void decTaskCounters(Collection<byte[]> regions, ServerName sn) {
1786     for (byte[] regBytes : regions) {
1787       AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
1788       regionCnt.decrementAndGet();
1789     }
1790 
1791     taskCounterPerServer.get(sn).decrementAndGet();
1792     tasksInProgress.decrementAndGet();
1793     synchronized (tasksInProgress) {
1794       tasksInProgress.notifyAll();
1795     }
1796   }
1797 
1798   /**
1799    * Creates the server error tracker to use inside process.
1800    * Currently, to preserve the main assumption about current retries, and to work well with
1801    * the retry-limit-based calculation, the calculation is local per Process object.
1802    * We may benefit from connection-wide tracking of server errors.
1803    * @return ServerErrorTracker to use, null if there is no ServerErrorTracker on this connection
1804    */
1805   protected ConnectionManager.ServerErrorTracker createServerErrorTracker() {
1806     return new ConnectionManager.ServerErrorTracker(
1807         this.serverTrackerTimeout, this.numTries);
1808   }
1809 
1810   private static boolean isReplicaGet(Row row) {
1811     return (row instanceof Get) && (((Get)row).getConsistency() == Consistency.TIMELINE);
1812   }
1813 
1814   /**
1815    * For manageError. Only used to make logging more clear, we don't actually care why we don't retry.
1816    */
1817   private enum Retry {
1818     YES,
1819     NO_LOCATION_PROBLEM,
1820     NO_NOT_RETRIABLE,
1821     NO_RETRIES_EXHAUSTED,
1822     NO_OTHER_SUCCEEDED
1823   }
1824 }