1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
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
106
107
108
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
122
123
124
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
132 public void waitUntilDone() throws InterruptedIOException;
133 }
134
135
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
151
152
153
154 private static class ReplicaResultState {
155 public ReplicaResultState(int callCount) {
156 this.callCount = callCount;
157 }
158
159
160 int callCount;
161
162
163 BatchErrors replicaErrors = null;
164
165 @Override
166 public String toString() {
167 return "[call count " + callCount + "; errors " + replicaErrors + "]";
168 }
169 }
170
171
172
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
188 private final int startLogErrorsCnt;
189
190
191
192
193 protected final int maxTotalConcurrentTasks;
194
195
196
197
198
199
200
201 protected final int maxConcurrentTasksPerRegion;
202
203
204
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
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
295
296
297
298
299
300
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
315
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
325
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
335
336
337
338
339
340
341
342
343
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();
358
359
360 List<Exception> locationErrors = null;
361 List<Integer> locationErrorRows = null;
362 do {
363
364 waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1, tableName.getNameAsString());
365
366
367
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
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
391
392 retainedActions.add(new Action<Row>(r, ++posInList));
393 locationErrors.add(ex);
394 locationErrorRows.add(posInList);
395 it.remove();
396 break;
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
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
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
439
440
441
442
443
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
461
462
463
464
465
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
475 return regionPrevious;
476 }
477
478 Boolean serverPrevious = serversIncluded.get(loc.getServerName());
479 if (Boolean.FALSE.equals(serverPrevious)) {
480
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
488 regionsIncluded.put(regionId, Boolean.FALSE);
489 return false;
490 }
491
492 if (serverPrevious == null) {
493
494 int newServers = 0;
495 for (Map.Entry<ServerName, Boolean> kv : serversIncluded.entrySet()) {
496 if (kv.getValue()) {
497 newServers++;
498 }
499 }
500
501
502 boolean ok = (newServers + tasksInProgress.get()) < maxTotalConcurrentTasks;
503
504 if (ok) {
505
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
528
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
537
538
539
540
541
542
543
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
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());
573 }
574
575
576
577
578
579
580
581
582
583
584 protected class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
585
586
587
588
589
590
591
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;
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
635 if (!actionsByServer.isEmpty()) {
636 sendMultiAction(actionsByServer, 1, null, true);
637 }
638 }
639 }
640
641
642
643
644
645
646 private void addReplicaActions(int index, Map<ServerName, MultiAction<Row>> actionsByServer,
647 List<Action<Row>> unknownReplicaActions) {
648 if (results[index] != null) return;
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
659
660
661 if (results[index] != null) return;
662
663
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
691
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
721 return;
722 }
723
724 } catch (IOException e) {
725
726
727 receiveGlobalFailure(multiAction, server, numAttempt, e);
728 return;
729 } catch (Throwable t) {
730
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
738 receiveMultiAction(multiAction, server, res, numAttempt);
739 } catch (Throwable t) {
740
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
763
764
765 private final Object replicaResultLock = new Object();
766
767
768
769
770
771
772
773 private final Object[] results;
774
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
801
802
803
804
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) {
812 if (replicaGetIndices == null) {
813 replicaGetIndices = new ArrayList<Integer>(actions.size() - 1);
814 }
815 replicaGetIndices.add(posInList);
816 }
817 } else if (!hasAnyNonReplicaReqs) {
818
819 hasAnyNonReplicaReqs = true;
820 if (posInList > 0) {
821
822
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
855
856
857
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
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
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
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
919
920 int replicaId = action.getReplicaId();
921 RegionLocations locs = findAllLocationsOrFail(action, true);
922 if (locs == null) return null;
923 HRegionLocation loc = locs.getRegionLocation(replicaId);
924 if (loc == null || loc.getServerName() == null) {
925 locs = findAllLocationsOrFail(action, false);
926 if (locs == null) return null;
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
968
969
970
971
972
973
974 private void sendMultiAction(Map<ServerName, MultiAction<Row>> actionsByServer,
975 int numAttempt, List<Action<Row>> actionsForReplicaThread, boolean reuseThread) {
976
977
978 int actionsRemaining = actionsByServer.size();
979
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
987
988 if (runnables.size() > actionsRemaining) {
989 actionsRemaining = runnables.size();
990 }
991
992
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
1002
1003 LOG.warn("#" + id + ", the task was rejected by the pool. This is unexpected." +
1004 " Server is " + server.getServerName(), t);
1005 } else {
1006
1007 LOG.warn("Caught unexpected exception/error: ", t);
1008 }
1009 decTaskCounters(multiAction.getRegions(), server);
1010
1011
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
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
1033 Map<Long, DelayingRunner> actions = new HashMap<Long, DelayingRunner>(multiAction
1034 .size());
1035
1036
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
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
1068
1069
1070
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
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
1088 replicaRunnable.run();
1089 } else {
1090
1091
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
1102
1103
1104
1105
1106
1107
1108
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
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
1128
1129
1130
1131
1132
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
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
1150
1151
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
1177
1178
1179 private void resubmit(ServerName oldServer, List<Action<Row>> toReplay,
1180 int numAttempt, int failureCount, Throwable throwable) {
1181
1182
1183
1184
1185
1186
1187
1188 long backOffTime = errorsByServer.calculateBackoffTime(oldServer, pause);
1189 if (numAttempt > startLogErrorsCnt) {
1190
1191
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
1215 LOG.warn(logMessage);
1216 } else {
1217 LOG.info(logMessage);
1218 }
1219 }
1220 }
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230 private void receiveMultiAction(MultiAction<Row> multiAction,
1231 ServerName server, MultiResponse responses, int numAttempt) {
1232 assert responses != null;
1233
1234
1235
1236
1237
1238
1239
1240 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
1241 Throwable throwable = null;
1242 int failureCount = 0;
1243 boolean canRetry = true;
1244
1245
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
1262 if (result == null || result instanceof Throwable) {
1263 Row row = sentAction.getAction();
1264
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
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
1287
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
1296
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
1309
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
1325
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
1389
1390
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;
1402 } else if ((state = trySetResultSimple(
1403 index, action.getAction(), false, result, null, isStale)) == null) {
1404 return;
1405 }
1406 assert state != null;
1407
1408
1409
1410
1411
1412 synchronized (state) {
1413 if (state.callCount == 0) return;
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
1428
1429
1430
1431
1432
1433 private void setError(int index, Row row, Throwable throwable, ServerName server) {
1434 ReplicaResultState state = null;
1435 if (results == null) {
1436
1437
1438
1439 errors.add(throwable, row, server);
1440 decActionCounter(index);
1441 return;
1442 } else if ((state = trySetResultSimple(
1443 index, row, true, throwable, server, false)) == null) {
1444 return;
1445 }
1446 assert state != null;
1447 BatchErrors target = null;
1448 boolean isActionDone = false;
1449 synchronized (state) {
1450 switch (state.callCount) {
1451 case 0: return;
1452 case 1: {
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) {
1471 errors.merge(state.replicaErrors);
1472 }
1473
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
1486
1487
1488
1489
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
1500
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
1525 errors.add((Throwable)result, row, server);
1526 }
1527
1528 if (resObj == null) {
1529
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) {
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
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
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
1653
1654 @VisibleForTesting
1655 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
1656 return rpcCallerFactory.<MultiResponse> newCaller();
1657 }
1658
1659 @VisibleForTesting
1660
1661 void waitUntilDone() throws InterruptedIOException {
1662 waitForMaximumCurrentTasks(0, null);
1663 }
1664
1665
1666 private void waitForMaximumCurrentTasks(int max, String tableName)
1667 throws InterruptedIOException {
1668 waitForMaximumCurrentTasks(max, tasksInProgress, id, tableName);
1669 }
1670
1671
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) {
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
1725
1726
1727
1728 public boolean hasError() {
1729 return globalErrors.hasErrors();
1730 }
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
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
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
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
1800
1801
1802
1803
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
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 }