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