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.concurrent.ConcurrentHashMap;
33 import java.util.concurrent.ConcurrentMap;
34 import java.util.concurrent.ConcurrentSkipListMap;
35 import java.util.concurrent.ExecutorService;
36 import java.util.concurrent.RejectedExecutionException;
37 import java.util.concurrent.TimeUnit;
38 import java.util.concurrent.atomic.AtomicInteger;
39 import java.util.concurrent.atomic.AtomicLong;
40
41 import org.apache.commons.logging.Log;
42 import org.apache.commons.logging.LogFactory;
43 import org.apache.hadoop.hbase.classification.InterfaceAudience;
44 import org.apache.hadoop.conf.Configuration;
45 import org.apache.hadoop.hbase.DoNotRetryIOException;
46 import org.apache.hadoop.hbase.HConstants;
47 import org.apache.hadoop.hbase.HRegionInfo;
48 import org.apache.hadoop.hbase.HRegionLocation;
49 import org.apache.hadoop.hbase.RegionLocations;
50 import org.apache.hadoop.hbase.ServerName;
51 import org.apache.hadoop.hbase.TableName;
52 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
53 import org.apache.hadoop.hbase.client.coprocessor.Batch;
54 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
55 import org.apache.hadoop.hbase.util.Bytes;
56 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
57 import org.apache.htrace.Trace;
58
59 import com.google.common.annotations.VisibleForTesting;
60
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 @InterfaceAudience.Private
97 class AsyncProcess {
98 protected static final Log LOG = LogFactory.getLog(AsyncProcess.class);
99 protected static final AtomicLong COUNTER = new AtomicLong();
100
101 public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget";
102
103
104
105
106
107
108
109 public static final String START_LOG_ERRORS_AFTER_COUNT_KEY =
110 "hbase.client.start.log.errors.counter";
111 public static final int DEFAULT_START_LOG_ERRORS_AFTER_COUNT = 9;
112
113
114
115
116
117
118
119 public static interface AsyncRequestFuture {
120 public boolean hasError();
121 public RetriesExhaustedWithDetailsException getErrors();
122 public List<? extends Row> getFailedOperations();
123 public Object[] getResults() throws InterruptedIOException;
124
125 public void waitUntilDone() throws InterruptedIOException;
126 }
127
128
129 private static final AsyncRequestFuture NO_REQS_RESULT = new AsyncRequestFuture() {
130 public final Object[] result = new Object[0];
131 @Override
132 public boolean hasError() { return false; }
133 @Override
134 public RetriesExhaustedWithDetailsException getErrors() { return null; }
135 @Override
136 public List<? extends Row> getFailedOperations() { return null; }
137 @Override
138 public Object[] getResults() { return result; }
139 @Override
140 public void waitUntilDone() throws InterruptedIOException {}
141 };
142
143
144
145
146
147 private static class ReplicaResultState {
148 public ReplicaResultState(int callCount) {
149 this.callCount = callCount;
150 }
151
152
153 int callCount;
154
155
156 BatchErrors replicaErrors = null;
157
158 @Override
159 public String toString() {
160 return "[call count " + callCount + "; errors " + replicaErrors + "]";
161 }
162 }
163
164
165
166 protected final long id;
167
168 protected final ClusterConnection connection;
169 protected final RpcRetryingCallerFactory rpcCallerFactory;
170 protected final RpcControllerFactory rpcFactory;
171 protected final BatchErrors globalErrors;
172 protected final ExecutorService pool;
173
174 protected final AtomicLong tasksInProgress = new AtomicLong(0);
175 protected final ConcurrentMap<byte[], AtomicInteger> taskCounterPerRegion =
176 new ConcurrentSkipListMap<byte[], AtomicInteger>(Bytes.BYTES_COMPARATOR);
177 protected final ConcurrentMap<ServerName, AtomicInteger> taskCounterPerServer =
178 new ConcurrentHashMap<ServerName, AtomicInteger>();
179
180
181 private final int startLogErrorsCnt;
182
183
184
185
186 protected final int maxTotalConcurrentTasks;
187
188
189
190
191
192
193
194 protected final int maxConcurrentTasksPerRegion;
195
196
197
198
199 protected final int maxConcurrentTasksPerServer;
200 protected final long pause;
201 protected int numTries;
202 protected int serverTrackerTimeout;
203 protected int timeout;
204 protected long primaryCallTimeoutMicroseconds;
205
206
207 protected static class BatchErrors {
208 private final List<Throwable> throwables = new ArrayList<Throwable>();
209 private final List<Row> actions = new ArrayList<Row>();
210 private final List<String> addresses = new ArrayList<String>();
211
212 public synchronized void add(Throwable ex, Row row, ServerName serverName) {
213 if (row == null){
214 throw new IllegalArgumentException("row cannot be null. location=" + serverName);
215 }
216
217 throwables.add(ex);
218 actions.add(row);
219 addresses.add(serverName != null ? serverName.toString() : "null");
220 }
221
222 public boolean hasErrors() {
223 return !throwables.isEmpty();
224 }
225
226 private synchronized RetriesExhaustedWithDetailsException makeException() {
227 return new RetriesExhaustedWithDetailsException(
228 new ArrayList<Throwable>(throwables),
229 new ArrayList<Row>(actions), new ArrayList<String>(addresses));
230 }
231
232 public synchronized void clear() {
233 throwables.clear();
234 actions.clear();
235 addresses.clear();
236 }
237
238 public synchronized void merge(BatchErrors other) {
239 throwables.addAll(other.throwables);
240 actions.addAll(other.actions);
241 addresses.addAll(other.addresses);
242 }
243 }
244
245 public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService pool,
246 RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors, RpcControllerFactory rpcFactory) {
247 if (hc == null) {
248 throw new IllegalArgumentException("HConnection cannot be null.");
249 }
250
251 this.connection = hc;
252 this.pool = pool;
253 this.globalErrors = useGlobalErrors ? new BatchErrors() : null;
254
255 this.id = COUNTER.incrementAndGet();
256
257 this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
258 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
259 this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
260 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
261 this.timeout = conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
262 HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
263 this.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000);
264
265 this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
266 HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS);
267 this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
268 HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS);
269 this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
270 HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS);
271
272 this.startLogErrorsCnt =
273 conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
274
275 if (this.maxTotalConcurrentTasks <= 0) {
276 throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
277 }
278 if (this.maxConcurrentTasksPerServer <= 0) {
279 throw new IllegalArgumentException("maxConcurrentTasksPerServer=" +
280 maxConcurrentTasksPerServer);
281 }
282 if (this.maxConcurrentTasksPerRegion <= 0) {
283 throw new IllegalArgumentException("maxConcurrentTasksPerRegion=" +
284 maxConcurrentTasksPerRegion);
285 }
286
287
288
289
290
291
292
293
294 this.serverTrackerTimeout = 0;
295 for (int i = 0; i < this.numTries; ++i) {
296 serverTrackerTimeout += ConnectionUtils.getPauseTime(this.pause, i);
297 }
298
299 this.rpcCallerFactory = rpcCaller;
300 this.rpcFactory = rpcFactory;
301 }
302
303
304
305
306
307 private ExecutorService getPool(ExecutorService pool) {
308 if (pool != null) return pool;
309 if (this.pool != null) return this.pool;
310 throw new RuntimeException("Neither AsyncProcess nor request have ExecutorService");
311 }
312
313
314
315
316
317 public <CResult> AsyncRequestFuture submit(TableName tableName, List<? extends Row> rows,
318 boolean atLeastOne, Batch.Callback<CResult> callback, boolean needResults)
319 throws InterruptedIOException {
320 return submit(null, tableName, rows, atLeastOne, callback, needResults);
321 }
322
323
324
325
326
327
328
329
330
331
332
333
334
335 public <CResult> AsyncRequestFuture submit(ExecutorService pool, TableName tableName,
336 List<? extends Row> rows, boolean atLeastOne, Batch.Callback<CResult> callback,
337 boolean needResults) throws InterruptedIOException {
338 if (rows.isEmpty()) {
339 return NO_REQS_RESULT;
340 }
341
342 Map<ServerName, MultiAction<Row>> actionsByServer =
343 new HashMap<ServerName, MultiAction<Row>>();
344 List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(rows.size());
345
346 NonceGenerator ng = this.connection.getNonceGenerator();
347 long nonceGroup = ng.getNonceGroup();
348
349
350 List<Exception> locationErrors = null;
351 List<Integer> locationErrorRows = null;
352 do {
353
354 waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1);
355
356
357
358 Map<Long, Boolean> regionIncluded = new HashMap<Long, Boolean>();
359 Map<ServerName, Boolean> serverIncluded = new HashMap<ServerName, Boolean>();
360
361 int posInList = -1;
362 Iterator<? extends Row> it = rows.iterator();
363 while (it.hasNext()) {
364 Row r = it.next();
365 HRegionLocation loc;
366 try {
367 if (r == null) throw new IllegalArgumentException("#" + id + ", row cannot be null");
368
369 RegionLocations locs = connection.locateRegion(
370 tableName, r.getRow(), true, true, RegionReplicaUtil.DEFAULT_REPLICA_ID);
371 if (locs == null || locs.isEmpty() || locs.getDefaultRegionLocation() == null) {
372 throw new IOException("#" + id + ", no location found, aborting submit for"
373 + " tableName=" + tableName + " rowkey=" + Bytes.toStringBinary(r.getRow()));
374 }
375 loc = locs.getDefaultRegionLocation();
376 } catch (IOException ex) {
377 locationErrors = new ArrayList<Exception>();
378 locationErrorRows = new ArrayList<Integer>();
379 LOG.error("Failed to get region location ", ex);
380
381
382 retainedActions.add(new Action<Row>(r, ++posInList));
383 locationErrors.add(ex);
384 locationErrorRows.add(posInList);
385 it.remove();
386 break;
387 }
388
389 if (canTakeOperation(loc, regionIncluded, serverIncluded)) {
390 Action<Row> action = new Action<Row>(r, ++posInList);
391 setNonce(ng, r, action);
392 retainedActions.add(action);
393
394 byte[] regionName = loc.getRegionInfo().getRegionName();
395 addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
396 it.remove();
397 }
398 }
399 } while (retainedActions.isEmpty() && atLeastOne && (locationErrors == null));
400
401 if (retainedActions.isEmpty()) return NO_REQS_RESULT;
402
403 return submitMultiActions(tableName, retainedActions, nonceGroup, callback, null, needResults,
404 locationErrors, locationErrorRows, actionsByServer, pool);
405 }
406
407 <CResult> AsyncRequestFuture submitMultiActions(TableName tableName,
408 List<Action<Row>> retainedActions, long nonceGroup, Batch.Callback<CResult> callback,
409 Object[] results, boolean needResults, List<Exception> locationErrors,
410 List<Integer> locationErrorRows, Map<ServerName, MultiAction<Row>> actionsByServer,
411 ExecutorService pool) {
412 AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(
413 tableName, retainedActions, nonceGroup, pool, callback, results, needResults);
414
415 if (locationErrors != null) {
416 for (int i = 0; i < locationErrors.size(); ++i) {
417 int originalIndex = locationErrorRows.get(i);
418 Row row = retainedActions.get(originalIndex).getAction();
419 ars.manageError(originalIndex, row,
420 Retry.NO_LOCATION_PROBLEM, locationErrors.get(i), null);
421 }
422 }
423 ars.sendMultiAction(actionsByServer, 1, null, false);
424 return ars;
425 }
426
427
428
429
430
431
432
433
434
435 private static void addAction(ServerName server, byte[] regionName, Action<Row> action,
436 Map<ServerName, MultiAction<Row>> actionsByServer, long nonceGroup) {
437 MultiAction<Row> multiAction = actionsByServer.get(server);
438 if (multiAction == null) {
439 multiAction = new MultiAction<Row>();
440 actionsByServer.put(server, multiAction);
441 }
442 if (action.hasNonce() && !multiAction.hasNonceGroup()) {
443 multiAction.setNonceGroup(nonceGroup);
444 }
445
446 multiAction.add(regionName, action);
447 }
448
449
450
451
452
453
454
455
456
457 protected boolean canTakeOperation(HRegionLocation loc,
458 Map<Long, Boolean> regionsIncluded,
459 Map<ServerName, Boolean> serversIncluded) {
460 long regionId = loc.getRegionInfo().getRegionId();
461 Boolean regionPrevious = regionsIncluded.get(regionId);
462
463 if (regionPrevious != null) {
464
465 return regionPrevious;
466 }
467
468 Boolean serverPrevious = serversIncluded.get(loc.getServerName());
469 if (Boolean.FALSE.equals(serverPrevious)) {
470
471 regionsIncluded.put(regionId, Boolean.FALSE);
472 return false;
473 }
474
475 AtomicInteger regionCnt = taskCounterPerRegion.get(loc.getRegionInfo().getRegionName());
476 if (regionCnt != null && regionCnt.get() >= maxConcurrentTasksPerRegion) {
477
478 regionsIncluded.put(regionId, Boolean.FALSE);
479 return false;
480 }
481
482 if (serverPrevious == null) {
483
484 int newServers = 0;
485 for (Map.Entry<ServerName, Boolean> kv : serversIncluded.entrySet()) {
486 if (kv.getValue()) {
487 newServers++;
488 }
489 }
490
491
492 boolean ok = (newServers + tasksInProgress.get()) < maxTotalConcurrentTasks;
493
494 if (ok) {
495
496 AtomicInteger serverCnt = taskCounterPerServer.get(loc.getServerName());
497 ok = (serverCnt == null || serverCnt.get() < maxConcurrentTasksPerServer);
498 }
499
500 if (!ok) {
501 regionsIncluded.put(regionId, Boolean.FALSE);
502 serversIncluded.put(loc.getServerName(), Boolean.FALSE);
503 return false;
504 }
505
506 serversIncluded.put(loc.getServerName(), Boolean.TRUE);
507 } else {
508 assert serverPrevious.equals(Boolean.TRUE);
509 }
510
511 regionsIncluded.put(regionId, Boolean.TRUE);
512
513 return true;
514 }
515
516
517
518
519
520 public <CResult> AsyncRequestFuture submitAll(TableName tableName,
521 List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results) {
522 return submitAll(null, tableName, rows, callback, results);
523 }
524
525
526
527
528
529
530
531
532
533
534
535 public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
536 List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results) {
537 List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
538
539
540 int posInList = -1;
541 NonceGenerator ng = this.connection.getNonceGenerator();
542 for (Row r : rows) {
543 posInList++;
544 if (r instanceof Put) {
545 Put put = (Put) r;
546 if (put.isEmpty()) {
547 throw new IllegalArgumentException("No columns to insert for #" + (posInList+1)+ " item");
548 }
549 }
550 Action<Row> action = new Action<Row>(r, posInList);
551 setNonce(ng, r, action);
552 actions.add(action);
553 }
554 AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(
555 tableName, actions, ng.getNonceGroup(), getPool(pool), callback, results, results != null);
556 ars.groupAndSendMultiAction(actions, 1);
557 return ars;
558 }
559
560 private static void setNonce(NonceGenerator ng, Row r, Action<Row> action) {
561 if (!(r instanceof Append) && !(r instanceof Increment)) return;
562 action.setNonce(ng.newNonce());
563 }
564
565
566
567
568
569
570
571
572
573
574 protected class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
575
576
577
578
579
580
581
582
583 private final class ReplicaCallIssuingRunnable implements Runnable {
584 private final long startTime;
585 private final List<Action<Row>> initialActions;
586
587 public ReplicaCallIssuingRunnable(List<Action<Row>> initialActions, long startTime) {
588 this.initialActions = initialActions;
589 this.startTime = startTime;
590 }
591
592 @Override
593 public void run() {
594 boolean done = false;
595 if (primaryCallTimeoutMicroseconds > 0) {
596 try {
597 done = waitUntilDone(startTime * 1000L + primaryCallTimeoutMicroseconds);
598 } catch (InterruptedException ex) {
599 LOG.error("Replica thread was interrupted - no replica calls: " + ex.getMessage());
600 return;
601 }
602 }
603 if (done) return;
604 Map<ServerName, MultiAction<Row>> actionsByServer =
605 new HashMap<ServerName, MultiAction<Row>>();
606 List<Action<Row>> unknownLocActions = new ArrayList<Action<Row>>();
607 if (replicaGetIndices == null) {
608 for (int i = 0; i < results.length; ++i) {
609 addReplicaActions(i, actionsByServer, unknownLocActions);
610 }
611 } else {
612 for (int replicaGetIndice : replicaGetIndices) {
613 addReplicaActions(replicaGetIndice, actionsByServer, unknownLocActions);
614 }
615 }
616 if (!actionsByServer.isEmpty()) {
617 sendMultiAction(actionsByServer, 1, null, unknownLocActions.isEmpty());
618 }
619 if (!unknownLocActions.isEmpty()) {
620 actionsByServer = new HashMap<ServerName, MultiAction<Row>>();
621 for (Action<Row> action : unknownLocActions) {
622 addReplicaActionsAgain(action, actionsByServer);
623 }
624
625 if (!actionsByServer.isEmpty()) {
626 sendMultiAction(actionsByServer, 1, null, true);
627 }
628 }
629 }
630
631
632
633
634
635
636 private void addReplicaActions(int index, Map<ServerName, MultiAction<Row>> actionsByServer,
637 List<Action<Row>> unknownReplicaActions) {
638 if (results[index] != null) return;
639 Action<Row> action = initialActions.get(index);
640 RegionLocations loc = findAllLocationsOrFail(action, true);
641 if (loc == null) return;
642 HRegionLocation[] locs = loc.getRegionLocations();
643 if (locs.length == 1) {
644 LOG.warn("No replicas found for " + action.getAction());
645 return;
646 }
647 synchronized (replicaResultLock) {
648
649
650
651 if (results[index] != null) return;
652
653
654 results[index] = new ReplicaResultState(locs.length);
655 }
656 for (int i = 1; i < locs.length; ++i) {
657 Action<Row> replicaAction = new Action<Row>(action, i);
658 if (locs[i] != null) {
659 addAction(locs[i].getServerName(), locs[i].getRegionInfo().getRegionName(),
660 replicaAction, actionsByServer, nonceGroup);
661 } else {
662 unknownReplicaActions.add(replicaAction);
663 }
664 }
665 }
666
667 private void addReplicaActionsAgain(
668 Action<Row> action, Map<ServerName, MultiAction<Row>> actionsByServer) {
669 if (action.getReplicaId() == RegionReplicaUtil.DEFAULT_REPLICA_ID) {
670 throw new AssertionError("Cannot have default replica here");
671 }
672 HRegionLocation loc = getReplicaLocationOrFail(action);
673 if (loc == null) return;
674 addAction(loc.getServerName(), loc.getRegionInfo().getRegionName(),
675 action, actionsByServer, nonceGroup);
676 }
677 }
678
679
680
681
682
683 private final class SingleServerRequestRunnable implements Runnable {
684 private final MultiAction<Row> multiAction;
685 private final int numAttempt;
686 private final ServerName server;
687
688 private SingleServerRequestRunnable(
689 MultiAction<Row> multiAction, int numAttempt, ServerName server) {
690 this.multiAction = multiAction;
691 this.numAttempt = numAttempt;
692 this.server = server;
693 }
694
695 @Override
696 public void run() {
697 MultiResponse res;
698 try {
699 MultiServerCallable<Row> callable = createCallable(server, tableName, multiAction);
700 try {
701 res = createCaller(callable).callWithoutRetries(callable, timeout);
702 } catch (IOException e) {
703
704
705 receiveGlobalFailure(multiAction, server, numAttempt, e);
706 return;
707 } catch (Throwable t) {
708
709 LOG.error("#" + id + ", Caught throwable while calling. This is unexpected." +
710 " Retrying. Server is " + server + ", tableName=" + tableName, t);
711 receiveGlobalFailure(multiAction, server, numAttempt, t);
712 return;
713 }
714
715
716 receiveMultiAction(multiAction, server, res, numAttempt);
717 } catch (Throwable t) {
718
719 LOG.error("Internal AsyncProcess #" + id + " error for "
720 + tableName + " processing for " + server, t);
721 throw new RuntimeException(t);
722 } finally {
723 decTaskCounters(multiAction.getRegions(), server);
724 }
725 }
726 }
727
728 private final Batch.Callback<CResult> callback;
729 private final BatchErrors errors;
730 private final ConnectionManager.ServerErrorTracker errorsByServer;
731 private final ExecutorService pool;
732
733
734 private final TableName tableName;
735 private final AtomicLong actionsInProgress = new AtomicLong(-1);
736
737
738
739 private final Object replicaResultLock = new Object();
740
741
742
743
744
745
746
747 private final Object[] results;
748
749 private final int[] replicaGetIndices;
750 private final boolean hasAnyReplicaGets;
751 private final long nonceGroup;
752
753 public AsyncRequestFutureImpl(TableName tableName, List<Action<Row>> actions, long nonceGroup,
754 ExecutorService pool, boolean needResults, Object[] results,
755 Batch.Callback<CResult> callback) {
756 this.pool = pool;
757 this.callback = callback;
758 this.nonceGroup = nonceGroup;
759 this.tableName = tableName;
760 this.actionsInProgress.set(actions.size());
761 if (results != null) {
762 assert needResults;
763 if (results.length != actions.size()) throw new AssertionError("results.length");
764 this.results = results;
765 for (int i = 0; i != this.results.length; ++i) {
766 results[i] = null;
767 }
768 } else {
769 this.results = needResults ? new Object[actions.size()] : null;
770 }
771 List<Integer> replicaGetIndices = null;
772 boolean hasAnyReplicaGets = false;
773 if (needResults) {
774
775
776
777
778
779 boolean hasAnyNonReplicaReqs = false;
780 int posInList = 0;
781 for (Action<Row> action : actions) {
782 boolean isReplicaGet = isReplicaGet(action.getAction());
783 if (isReplicaGet) {
784 hasAnyReplicaGets = true;
785 if (hasAnyNonReplicaReqs) {
786 if (replicaGetIndices == null) {
787 replicaGetIndices = new ArrayList<Integer>(actions.size() - 1);
788 }
789 replicaGetIndices.add(posInList);
790 }
791 } else if (!hasAnyNonReplicaReqs) {
792
793 hasAnyNonReplicaReqs = true;
794 if (posInList > 0) {
795
796
797 replicaGetIndices = new ArrayList<Integer>(actions.size() - 1);
798 for (int i = 0; i < posInList; ++i) {
799 replicaGetIndices.add(i);
800 }
801 }
802 }
803 ++posInList;
804 }
805 }
806 this.hasAnyReplicaGets = hasAnyReplicaGets;
807 if (replicaGetIndices != null) {
808 this.replicaGetIndices = new int[replicaGetIndices.size()];
809 int i = 0;
810 for (Integer el : replicaGetIndices) {
811 this.replicaGetIndices[i++] = el;
812 }
813 } else {
814 this.replicaGetIndices = null;
815 }
816 this.errorsByServer = createServerErrorTracker();
817 this.errors = (globalErrors != null) ? globalErrors : new BatchErrors();
818 }
819
820
821
822
823
824
825
826 private void groupAndSendMultiAction(List<Action<Row>> currentActions, int numAttempt) {
827 Map<ServerName, MultiAction<Row>> actionsByServer =
828 new HashMap<ServerName, MultiAction<Row>>();
829
830 boolean isReplica = false;
831 List<Action<Row>> unknownReplicaActions = null;
832 for (Action<Row> action : currentActions) {
833 RegionLocations locs = findAllLocationsOrFail(action, true);
834 if (locs == null) continue;
835 boolean isReplicaAction = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
836 if (isReplica && !isReplicaAction) {
837
838 throw new AssertionError("Replica and non-replica actions in the same retry");
839 }
840 isReplica = isReplicaAction;
841 HRegionLocation loc = locs.getRegionLocation(action.getReplicaId());
842 if (loc == null || loc.getServerName() == null) {
843 if (isReplica) {
844 if (unknownReplicaActions == null) {
845 unknownReplicaActions = new ArrayList<Action<Row>>();
846 }
847 unknownReplicaActions.add(action);
848 } else {
849
850 manageLocationError(action, null);
851 }
852 } else {
853 byte[] regionName = loc.getRegionInfo().getRegionName();
854 addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
855 }
856 }
857 boolean doStartReplica = (numAttempt == 1 && !isReplica && hasAnyReplicaGets);
858 boolean hasUnknown = unknownReplicaActions != null && !unknownReplicaActions.isEmpty();
859
860 if (!actionsByServer.isEmpty()) {
861
862 sendMultiAction(actionsByServer, numAttempt, (doStartReplica && !hasUnknown)
863 ? currentActions : null, numAttempt > 1 && !hasUnknown);
864 }
865
866 if (hasUnknown) {
867 actionsByServer = new HashMap<ServerName, MultiAction<Row>>();
868 for (Action<Row> action : unknownReplicaActions) {
869 HRegionLocation loc = getReplicaLocationOrFail(action);
870 if (loc == null) continue;
871 byte[] regionName = loc.getRegionInfo().getRegionName();
872 addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
873 }
874 if (!actionsByServer.isEmpty()) {
875 sendMultiAction(
876 actionsByServer, numAttempt, doStartReplica ? currentActions : null, true);
877 }
878 }
879 }
880
881 private HRegionLocation getReplicaLocationOrFail(Action<Row> action) {
882
883
884 int replicaId = action.getReplicaId();
885 RegionLocations locs = findAllLocationsOrFail(action, true);
886 if (locs == null) return null;
887 HRegionLocation loc = locs.getRegionLocation(replicaId);
888 if (loc == null || loc.getServerName() == null) {
889 locs = findAllLocationsOrFail(action, false);
890 if (locs == null) return null;
891 loc = locs.getRegionLocation(replicaId);
892 }
893 if (loc == null || loc.getServerName() == null) {
894 manageLocationError(action, null);
895 return null;
896 }
897 return loc;
898 }
899
900 private void manageLocationError(Action<Row> action, Exception ex) {
901 String msg = "Cannot get replica " + action.getReplicaId()
902 + " location for " + action.getAction();
903 LOG.error(msg);
904 if (ex == null) {
905 ex = new IOException(msg);
906 }
907 manageError(action.getOriginalIndex(), action.getAction(),
908 Retry.NO_LOCATION_PROBLEM, ex, null);
909 }
910
911 private RegionLocations findAllLocationsOrFail(Action<Row> action, boolean useCache) {
912 if (action.getAction() == null) throw new IllegalArgumentException("#" + id +
913 ", row cannot be null");
914 RegionLocations loc = null;
915 try {
916 loc = connection.locateRegion(
917 tableName, action.getAction().getRow(), useCache, true, action.getReplicaId());
918 } catch (IOException ex) {
919 manageLocationError(action, ex);
920 }
921 return loc;
922 }
923
924
925
926
927
928
929
930
931
932 private void sendMultiAction(Map<ServerName, MultiAction<Row>> actionsByServer,
933 int numAttempt, List<Action<Row>> actionsForReplicaThread, boolean reuseThread) {
934
935
936 int actionsRemaining = actionsByServer.size();
937
938 for (Map.Entry<ServerName, MultiAction<Row>> e : actionsByServer.entrySet()) {
939 ServerName server = e.getKey();
940 MultiAction<Row> multiAction = e.getValue();
941 incTaskCounters(multiAction.getRegions(), server);
942 Collection<? extends Runnable> runnables = getNewMultiActionRunnable(server, multiAction,
943 numAttempt);
944
945
946 if (runnables.size() > actionsRemaining) {
947 actionsRemaining = runnables.size();
948 }
949
950
951 for (Runnable runnable : runnables) {
952 if ((--actionsRemaining == 0) && reuseThread) {
953 runnable.run();
954 } else {
955 try {
956 pool.submit(runnable);
957 } catch (RejectedExecutionException ree) {
958
959
960 decTaskCounters(multiAction.getRegions(), server);
961 LOG.warn("#" + id + ", the task was rejected by the pool. This is unexpected." +
962 " Server is " + server.getServerName(), ree);
963
964
965 receiveGlobalFailure(multiAction, server, numAttempt, ree);
966 }
967 }
968 }
969 }
970
971 if (actionsForReplicaThread != null) {
972 startWaitingForReplicaCalls(actionsForReplicaThread);
973 }
974 }
975
976 private Collection<? extends Runnable> getNewMultiActionRunnable(ServerName server,
977 MultiAction<Row> multiAction,
978 int numAttempt) {
979
980 if (AsyncProcess.this.connection.getStatisticsTracker() == null) {
981 return Collections.singletonList(Trace.wrap("AsyncProcess.sendMultiAction",
982 new SingleServerRequestRunnable(multiAction, numAttempt, server)));
983 }
984
985
986 Map<Long, DelayingRunner> actions = new HashMap<Long, DelayingRunner>(multiAction
987 .size());
988
989
990 for (Map.Entry<byte[], List<Action<Row>>> e : multiAction.actions.entrySet()) {
991 Long backoff = getBackoff(server, e.getKey());
992 DelayingRunner runner = actions.get(backoff);
993 if (runner == null) {
994 actions.put(backoff, new DelayingRunner(backoff, e));
995 } else {
996 runner.add(e);
997 }
998 }
999
1000 List<Runnable> toReturn = new ArrayList<Runnable>(actions.size());
1001 for (DelayingRunner runner : actions.values()) {
1002 String traceText = "AsyncProcess.sendMultiAction";
1003 Runnable runnable =
1004 new SingleServerRequestRunnable(runner.getActions(), numAttempt, server);
1005
1006 if (runner.getSleepTime() > 0) {
1007 runner.setRunner(runnable);
1008 traceText = "AsyncProcess.clientBackoff.sendMultiAction";
1009 runnable = runner;
1010 }
1011 runnable = Trace.wrap(traceText, runnable);
1012 toReturn.add(runnable);
1013
1014 }
1015 return toReturn;
1016 }
1017
1018
1019
1020
1021
1022
1023
1024 private Long getBackoff(ServerName server, byte[] regionName) {
1025 ServerStatisticTracker tracker = AsyncProcess.this.connection.getStatisticsTracker();
1026 ServerStatistics stats = tracker.getStats(server);
1027 return AsyncProcess.this.connection.getBackoffPolicy()
1028 .getBackoffTime(server, regionName, stats);
1029 }
1030
1031
1032
1033
1034 private void startWaitingForReplicaCalls(List<Action<Row>> actionsForReplicaThread) {
1035 long startTime = EnvironmentEdgeManager.currentTime();
1036 ReplicaCallIssuingRunnable replicaRunnable = new ReplicaCallIssuingRunnable(
1037 actionsForReplicaThread, startTime);
1038 if (primaryCallTimeoutMicroseconds == 0) {
1039
1040 replicaRunnable.run();
1041 } else {
1042
1043
1044 try {
1045 pool.submit(replicaRunnable);
1046 } catch (RejectedExecutionException ree) {
1047 LOG.warn("#" + id + ", replica task was rejected by the pool - no replica calls", ree);
1048 }
1049 }
1050 }
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062 public Retry manageError(int originalIndex, Row row, Retry canRetry,
1063 Throwable throwable, ServerName server) {
1064 if (canRetry == Retry.YES
1065 && throwable != null && throwable instanceof DoNotRetryIOException) {
1066 canRetry = Retry.NO_NOT_RETRIABLE;
1067 }
1068
1069 if (canRetry != Retry.YES) {
1070
1071 setError(originalIndex, row, throwable, server);
1072 } else if (isActionComplete(originalIndex, row)) {
1073 canRetry = Retry.NO_OTHER_SUCCEEDED;
1074 }
1075 return canRetry;
1076 }
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086 private void receiveGlobalFailure(
1087 MultiAction<Row> rsActions, ServerName server, int numAttempt, Throwable t) {
1088 errorsByServer.reportServerError(server);
1089 Retry canRetry = errorsByServer.canRetryMore(numAttempt)
1090 ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED;
1091
1092 if (tableName == null) {
1093
1094 connection.clearCaches(server);
1095 }
1096 int failed = 0, stopped = 0;
1097 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
1098 for (Map.Entry<byte[], List<Action<Row>>> e : rsActions.actions.entrySet()) {
1099 byte[] regionName = e.getKey();
1100 byte[] row = e.getValue().iterator().next().getAction().getRow();
1101
1102
1103
1104 if (tableName != null) {
1105 connection.updateCachedLocations(tableName, regionName, row, null, server);
1106 }
1107 for (Action<Row> action : e.getValue()) {
1108 Retry retry = manageError(
1109 action.getOriginalIndex(), action.getAction(), canRetry, t, server);
1110 if (retry == Retry.YES) {
1111 toReplay.add(action);
1112 } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
1113 ++stopped;
1114 } else {
1115 ++failed;
1116 }
1117 }
1118 }
1119
1120 if (toReplay.isEmpty()) {
1121 logNoResubmit(server, numAttempt, rsActions.size(), t, failed, stopped);
1122 } else {
1123 resubmit(server, toReplay, numAttempt, rsActions.size(), t);
1124 }
1125 }
1126
1127
1128
1129
1130
1131 private void resubmit(ServerName oldServer, List<Action<Row>> toReplay,
1132 int numAttempt, int failureCount, Throwable throwable) {
1133
1134
1135
1136
1137
1138
1139
1140 long backOffTime = errorsByServer.calculateBackoffTime(oldServer, pause);
1141 if (numAttempt > startLogErrorsCnt) {
1142
1143
1144 LOG.info(createLog(numAttempt, failureCount, toReplay.size(),
1145 oldServer, throwable, backOffTime, true, null, -1, -1));
1146 }
1147
1148 try {
1149 Thread.sleep(backOffTime);
1150 } catch (InterruptedException e) {
1151 LOG.warn("#" + id + ", not sent: " + toReplay.size() + " operations, " + oldServer, e);
1152 Thread.currentThread().interrupt();
1153 return;
1154 }
1155
1156 groupAndSendMultiAction(toReplay, numAttempt + 1);
1157 }
1158
1159 private void logNoResubmit(ServerName oldServer, int numAttempt,
1160 int failureCount, Throwable throwable, int failed, int stopped) {
1161 if (failureCount != 0 || numAttempt > startLogErrorsCnt + 1) {
1162 String timeStr = new Date(errorsByServer.getStartTrackingTime()).toString();
1163 String logMessage = createLog(numAttempt, failureCount, 0, oldServer,
1164 throwable, -1, false, timeStr, failed, stopped);
1165 if (failed != 0) {
1166
1167 LOG.warn(logMessage);
1168 } else {
1169 LOG.info(logMessage);
1170 }
1171 }
1172 }
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182 private void receiveMultiAction(MultiAction<Row> multiAction,
1183 ServerName server, MultiResponse responses, int numAttempt) {
1184 assert responses != null;
1185
1186
1187
1188
1189
1190
1191
1192 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
1193 Throwable throwable = null;
1194 int failureCount = 0;
1195 boolean canRetry = true;
1196
1197
1198 int failed = 0, stopped = 0;
1199 for (Map.Entry<byte[], List<Action<Row>>> regionEntry : multiAction.actions.entrySet()) {
1200 byte[] regionName = regionEntry.getKey();
1201 Map<Integer, Object> regionResults = responses.getResults().get(regionName);
1202 if (regionResults == null) {
1203 if (!responses.getExceptions().containsKey(regionName)) {
1204 LOG.error("Server sent us neither results nor exceptions for "
1205 + Bytes.toStringBinary(regionName));
1206 responses.getExceptions().put(regionName, new RuntimeException("Invalid response"));
1207 }
1208 continue;
1209 }
1210 boolean regionFailureRegistered = false;
1211 for (Action<Row> sentAction : regionEntry.getValue()) {
1212 Object result = regionResults.get(sentAction.getOriginalIndex());
1213
1214 if (result == null || result instanceof Throwable) {
1215 Row row = sentAction.getAction();
1216
1217 if (!regionFailureRegistered) {
1218 regionFailureRegistered = true;
1219 connection.updateCachedLocations(
1220 tableName, regionName, row.getRow(), result, server);
1221 }
1222 if (failureCount == 0) {
1223 errorsByServer.reportServerError(server);
1224
1225 canRetry = errorsByServer.canRetryMore(numAttempt);
1226 }
1227 ++failureCount;
1228 Retry retry = manageError(sentAction.getOriginalIndex(), row,
1229 canRetry ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED, (Throwable)result, server);
1230 if (retry == Retry.YES) {
1231 toReplay.add(sentAction);
1232 } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
1233 ++stopped;
1234 } else {
1235 ++failed;
1236 }
1237 } else {
1238
1239
1240 if (AsyncProcess.this.connection.getStatisticsTracker() != null) {
1241 result = ResultStatsUtil.updateStats(result,
1242 AsyncProcess.this.connection.getStatisticsTracker(), server, regionName);
1243 }
1244
1245 if (callback != null) {
1246 try {
1247
1248
1249 this.callback.update(regionName, sentAction.getAction().getRow(), (CResult)result);
1250 } catch (Throwable t) {
1251 LOG.error("User callback threw an exception for "
1252 + Bytes.toStringBinary(regionName) + ", ignoring", t);
1253 }
1254 }
1255 setResult(sentAction, result);
1256 }
1257 }
1258 }
1259
1260
1261
1262 for (Map.Entry<byte[], Throwable> throwableEntry : responses.getExceptions().entrySet()) {
1263 throwable = throwableEntry.getValue();
1264 byte[] region = throwableEntry.getKey();
1265 List<Action<Row>> actions = multiAction.actions.get(region);
1266 if (actions == null || actions.isEmpty()) {
1267 throw new IllegalStateException("Wrong response for the region: " +
1268 HRegionInfo.encodeRegionName(region));
1269 }
1270
1271 if (failureCount == 0) {
1272 errorsByServer.reportServerError(server);
1273 canRetry = errorsByServer.canRetryMore(numAttempt);
1274 }
1275 connection.updateCachedLocations(
1276 tableName, region, actions.get(0).getAction().getRow(), throwable, server);
1277 failureCount += actions.size();
1278
1279 for (Action<Row> action : actions) {
1280 Row row = action.getAction();
1281 Retry retry = manageError(action.getOriginalIndex(), row,
1282 canRetry ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED, throwable, server);
1283 if (retry == Retry.YES) {
1284 toReplay.add(action);
1285 } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
1286 ++stopped;
1287 } else {
1288 ++failed;
1289 }
1290 }
1291 }
1292
1293 if (toReplay.isEmpty()) {
1294 logNoResubmit(server, numAttempt, failureCount, throwable, failed, stopped);
1295 } else {
1296 resubmit(server, toReplay, numAttempt, failureCount, throwable);
1297 }
1298 }
1299
1300 private String createLog(int numAttempt, int failureCount, int replaySize, ServerName sn,
1301 Throwable error, long backOffTime, boolean willRetry, String startTime,
1302 int failed, int stopped) {
1303 StringBuilder sb = new StringBuilder();
1304 sb.append("#").append(id).append(", table=").append(tableName).append(", ")
1305 .append("attempt=").append(numAttempt)
1306 .append("/").append(numTries).append(" ");
1307
1308 if (failureCount > 0 || error != null){
1309 sb.append("failed=").append(failureCount).append("ops").append(", last exception: ").
1310 append(error == null ? "null" : error);
1311 } else {
1312 sb.append("succeeded");
1313 }
1314
1315 sb.append(" on ").append(sn).append(", tracking started ").append(startTime);
1316
1317 if (willRetry) {
1318 sb.append(", retrying after=").append(backOffTime).append("ms").
1319 append(", replay=").append(replaySize).append("ops");
1320 } else if (failureCount > 0) {
1321 if (stopped > 0) {
1322 sb.append("; not retrying ").append(stopped).append(" due to success from other replica");
1323 }
1324 if (failed > 0) {
1325 sb.append("; not retrying ").append(failed).append(" - final failure");
1326 }
1327
1328 }
1329
1330 return sb.toString();
1331 }
1332
1333
1334
1335
1336
1337
1338 private void setResult(Action<Row> action, Object result) {
1339 if (result == null) {
1340 throw new RuntimeException("Result cannot be null");
1341 }
1342 ReplicaResultState state = null;
1343 boolean isStale = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
1344 int index = action.getOriginalIndex();
1345 if (results == null) {
1346 decActionCounter(index);
1347 return;
1348 } else if ((state = trySetResultSimple(
1349 index, action.getAction(), false, result, null, isStale)) == null) {
1350 return;
1351 }
1352 assert state != null;
1353
1354
1355
1356
1357
1358 synchronized (state) {
1359 if (state.callCount == 0) return;
1360 state.callCount = 0;
1361 }
1362 synchronized (replicaResultLock) {
1363 if (results[index] != state) {
1364 throw new AssertionError("We set the callCount but someone else replaced the result");
1365 }
1366 results[index] = result;
1367 }
1368
1369 decActionCounter(index);
1370 }
1371
1372
1373
1374
1375
1376
1377
1378
1379 private void setError(int index, Row row, Throwable throwable, ServerName server) {
1380 ReplicaResultState state = null;
1381 if (results == null) {
1382
1383
1384
1385 errors.add(throwable, row, server);
1386 decActionCounter(index);
1387 return;
1388 } else if ((state = trySetResultSimple(
1389 index, row, true, throwable, server, false)) == null) {
1390 return;
1391 }
1392 assert state != null;
1393 BatchErrors target = null;
1394 boolean isActionDone = false;
1395 synchronized (state) {
1396 switch (state.callCount) {
1397 case 0: return;
1398 case 1: {
1399 target = errors;
1400 isActionDone = true;
1401 break;
1402 }
1403 default: {
1404 assert state.callCount > 1;
1405 if (state.replicaErrors == null) {
1406 state.replicaErrors = new BatchErrors();
1407 }
1408 target = state.replicaErrors;
1409 break;
1410 }
1411 }
1412 --state.callCount;
1413 }
1414 target.add(throwable, row, server);
1415 if (isActionDone) {
1416 if (state.replicaErrors != null) {
1417 errors.merge(state.replicaErrors);
1418 }
1419
1420 synchronized (replicaResultLock) {
1421 if (results[index] != state) {
1422 throw new AssertionError("We set the callCount but someone else replaced the result");
1423 }
1424 results[index] = throwable;
1425 }
1426 decActionCounter(index);
1427 }
1428 }
1429
1430
1431
1432
1433
1434
1435
1436
1437 private boolean isActionComplete(int index, Row row) {
1438 if (!isReplicaGet(row)) return false;
1439 Object resObj = results[index];
1440 return (resObj != null) && (!(resObj instanceof ReplicaResultState)
1441 || ((ReplicaResultState)resObj).callCount == 0);
1442 }
1443
1444
1445
1446
1447
1448 private ReplicaResultState trySetResultSimple(int index, Row row, boolean isError,
1449 Object result, ServerName server, boolean isFromReplica) {
1450 Object resObj = null;
1451 if (!isReplicaGet(row)) {
1452 if (isFromReplica) {
1453 throw new AssertionError("Unexpected stale result for " + row);
1454 }
1455 results[index] = result;
1456 } else {
1457 synchronized (replicaResultLock) {
1458 if ((resObj = results[index]) == null) {
1459 if (isFromReplica) {
1460 throw new AssertionError("Unexpected stale result for " + row);
1461 }
1462 results[index] = result;
1463 }
1464 }
1465 }
1466
1467 ReplicaResultState rrs =
1468 (resObj instanceof ReplicaResultState) ? (ReplicaResultState)resObj : null;
1469 if (rrs == null && isError) {
1470
1471 errors.add((Throwable)result, row, server);
1472 }
1473
1474 if (resObj == null) {
1475
1476 decActionCounter(index);
1477 return null;
1478 }
1479 return rrs;
1480 }
1481
1482 private void decActionCounter(int index) {
1483 long actionsRemaining = actionsInProgress.decrementAndGet();
1484 if (actionsRemaining < 0) {
1485 String error = buildDetailedErrorMsg("Incorrect actions in progress", index);
1486 throw new AssertionError(error);
1487 } else if (actionsRemaining == 0) {
1488 synchronized (actionsInProgress) {
1489 actionsInProgress.notifyAll();
1490 }
1491 }
1492 }
1493
1494 private String buildDetailedErrorMsg(String string, int index) {
1495 String error = string + "; called for " + index +
1496 ", actionsInProgress " + actionsInProgress.get() + "; replica gets: ";
1497 if (replicaGetIndices != null) {
1498 for (int i = 0; i < replicaGetIndices.length; ++i) {
1499 error += replicaGetIndices[i] + ", ";
1500 }
1501 } else {
1502 error += (hasAnyReplicaGets ? "all" : "none");
1503 }
1504 error += "; results ";
1505 if (results != null) {
1506 for (int i = 0; i < results.length; ++i) {
1507 Object o = results[i];
1508 error += ((o == null) ? "null" : o.toString()) + ", ";
1509 }
1510 }
1511 return error;
1512 }
1513
1514 @Override
1515 public void waitUntilDone() throws InterruptedIOException {
1516 try {
1517 waitUntilDone(Long.MAX_VALUE);
1518 } catch (InterruptedException iex) {
1519 throw new InterruptedIOException(iex.getMessage());
1520 }
1521 }
1522
1523 private boolean waitUntilDone(long cutoff) throws InterruptedException {
1524 boolean hasWait = cutoff != Long.MAX_VALUE;
1525 long lastLog = EnvironmentEdgeManager.currentTime();
1526 long currentInProgress;
1527 while (0 != (currentInProgress = actionsInProgress.get())) {
1528 long now = EnvironmentEdgeManager.currentTime();
1529 if (hasWait && (now * 1000L) > cutoff) {
1530 return false;
1531 }
1532 if (!hasWait) {
1533 if (now > lastLog + 10000) {
1534 lastLog = now;
1535 LOG.info("#" + id + ", waiting for " + currentInProgress + " actions to finish");
1536 }
1537 }
1538 synchronized (actionsInProgress) {
1539 if (actionsInProgress.get() == 0) break;
1540 if (!hasWait) {
1541 actionsInProgress.wait(100);
1542 } else {
1543 long waitMicroSecond = Math.min(100000L, (cutoff - now * 1000L));
1544 TimeUnit.MICROSECONDS.timedWait(actionsInProgress, waitMicroSecond);
1545 }
1546 }
1547 }
1548 return true;
1549 }
1550
1551 @Override
1552 public boolean hasError() {
1553 return errors.hasErrors();
1554 }
1555
1556 @Override
1557 public List<? extends Row> getFailedOperations() {
1558 return errors.actions;
1559 }
1560
1561 @Override
1562 public RetriesExhaustedWithDetailsException getErrors() {
1563 return errors.makeException();
1564 }
1565
1566 @Override
1567 public Object[] getResults() throws InterruptedIOException {
1568 waitUntilDone();
1569 return results;
1570 }
1571 }
1572
1573 @VisibleForTesting
1574
1575 protected <CResult> AsyncRequestFutureImpl<CResult> createAsyncRequestFuture(
1576 TableName tableName, List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
1577 Batch.Callback<CResult> callback, Object[] results, boolean needResults) {
1578 return new AsyncRequestFutureImpl<CResult>(
1579 tableName, actions, nonceGroup, getPool(pool), needResults, results, callback);
1580 }
1581
1582
1583
1584
1585 @VisibleForTesting
1586 protected MultiServerCallable<Row> createCallable(final ServerName server,
1587 TableName tableName, final MultiAction<Row> multi) {
1588 return new MultiServerCallable<Row>(connection, tableName, server, this.rpcFactory, multi);
1589 }
1590
1591
1592
1593
1594 @VisibleForTesting
1595 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
1596 return rpcCallerFactory.<MultiResponse> newCaller();
1597 }
1598
1599 @VisibleForTesting
1600
1601 void waitUntilDone() throws InterruptedIOException {
1602 waitForMaximumCurrentTasks(0);
1603 }
1604
1605
1606 private void waitForMaximumCurrentTasks(int max) throws InterruptedIOException {
1607 long lastLog = EnvironmentEdgeManager.currentTime();
1608 long currentInProgress, oldInProgress = Long.MAX_VALUE;
1609 while ((currentInProgress = this.tasksInProgress.get()) > max) {
1610 if (oldInProgress != currentInProgress) {
1611 long now = EnvironmentEdgeManager.currentTime();
1612 if (now > lastLog + 10000) {
1613 lastLog = now;
1614 LOG.info("#" + id + ", waiting for some tasks to finish. Expected max="
1615 + max + ", tasksInProgress=" + currentInProgress);
1616 }
1617 }
1618 oldInProgress = currentInProgress;
1619 try {
1620 synchronized (this.tasksInProgress) {
1621 if (tasksInProgress.get() != oldInProgress) break;
1622 this.tasksInProgress.wait(100);
1623 }
1624 } catch (InterruptedException e) {
1625 throw new InterruptedIOException("#" + id + ", interrupted." +
1626 " currentNumberOfTask=" + currentInProgress);
1627 }
1628 }
1629 }
1630
1631
1632
1633
1634
1635
1636 public boolean hasError() {
1637 return globalErrors.hasErrors();
1638 }
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649 public RetriesExhaustedWithDetailsException waitForAllPreviousOpsAndReset(
1650 List<Row> failedRows) throws InterruptedIOException {
1651 waitForMaximumCurrentTasks(0);
1652 if (!globalErrors.hasErrors()) {
1653 return null;
1654 }
1655 if (failedRows != null) {
1656 failedRows.addAll(globalErrors.actions);
1657 }
1658 RetriesExhaustedWithDetailsException result = globalErrors.makeException();
1659 globalErrors.clear();
1660 return result;
1661 }
1662
1663
1664
1665
1666 protected void incTaskCounters(Collection<byte[]> regions, ServerName sn) {
1667 tasksInProgress.incrementAndGet();
1668
1669 AtomicInteger serverCnt = taskCounterPerServer.get(sn);
1670 if (serverCnt == null) {
1671 taskCounterPerServer.putIfAbsent(sn, new AtomicInteger());
1672 serverCnt = taskCounterPerServer.get(sn);
1673 }
1674 serverCnt.incrementAndGet();
1675
1676 for (byte[] regBytes : regions) {
1677 AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
1678 if (regionCnt == null) {
1679 regionCnt = new AtomicInteger();
1680 AtomicInteger oldCnt = taskCounterPerRegion.putIfAbsent(regBytes, regionCnt);
1681 if (oldCnt != null) {
1682 regionCnt = oldCnt;
1683 }
1684 }
1685 regionCnt.incrementAndGet();
1686 }
1687 }
1688
1689
1690
1691
1692 protected void decTaskCounters(Collection<byte[]> regions, ServerName sn) {
1693 for (byte[] regBytes : regions) {
1694 AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
1695 regionCnt.decrementAndGet();
1696 }
1697
1698 taskCounterPerServer.get(sn).decrementAndGet();
1699 tasksInProgress.decrementAndGet();
1700 synchronized (tasksInProgress) {
1701 tasksInProgress.notifyAll();
1702 }
1703 }
1704
1705
1706
1707
1708
1709
1710
1711
1712 protected ConnectionManager.ServerErrorTracker createServerErrorTracker() {
1713 return new ConnectionManager.ServerErrorTracker(
1714 this.serverTrackerTimeout, this.numTries);
1715 }
1716
1717 private static boolean isReplicaGet(Row row) {
1718 return (row instanceof Get) && (((Get)row).getConsistency() == Consistency.TIMELINE);
1719 }
1720
1721
1722
1723
1724 private enum Retry {
1725 YES,
1726 NO_LOCATION_PROBLEM,
1727 NO_NOT_RETRIABLE,
1728 NO_RETRIES_EXHAUSTED,
1729 NO_OTHER_SUCCEEDED
1730 }
1731 }