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.Arrays;
26 import java.util.Collection;
27 import java.util.Collections;
28 import java.util.Date;
29 import java.util.HashMap;
30 import java.util.HashSet;
31 import java.util.Iterator;
32 import java.util.List;
33 import java.util.Map;
34 import java.util.Set;
35 import java.util.TreeSet;
36 import java.util.concurrent.ConcurrentHashMap;
37 import java.util.concurrent.ConcurrentMap;
38 import java.util.concurrent.ConcurrentSkipListMap;
39 import java.util.concurrent.ExecutorService;
40 import java.util.concurrent.RejectedExecutionException;
41 import java.util.concurrent.TimeUnit;
42 import java.util.concurrent.atomic.AtomicInteger;
43 import java.util.concurrent.atomic.AtomicLong;
44 import org.apache.commons.logging.Log;
45 import org.apache.commons.logging.LogFactory;
46 import org.apache.hadoop.conf.Configuration;
47 import org.apache.hadoop.hbase.client.AsyncProcess.RowChecker.ReturnCode;
48 import org.apache.hadoop.hbase.CallQueueTooBigException;
49 import org.apache.hadoop.hbase.DoNotRetryIOException;
50 import org.apache.hadoop.hbase.HConstants;
51 import org.apache.hadoop.hbase.HRegionInfo;
52 import org.apache.hadoop.hbase.HRegionLocation;
53 import org.apache.hadoop.hbase.RegionLocations;
54 import org.apache.hadoop.hbase.RetryImmediatelyException;
55 import org.apache.hadoop.hbase.ServerName;
56 import org.apache.hadoop.hbase.TableName;
57 import org.apache.hadoop.hbase.classification.InterfaceAudience;
58 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
59 import org.apache.hadoop.hbase.client.coprocessor.Batch;
60 import org.apache.hadoop.hbase.exceptions.ClientExceptionsUtil;
61 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
62 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
63 import org.apache.hadoop.hbase.util.Bytes;
64 import org.apache.hadoop.hbase.util.EnvironmentEdge;
65 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
66 import org.apache.htrace.Trace;
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
98
99
100
101
102
103 @InterfaceAudience.Private
104 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="JLM_JSR166_UTILCONCURRENT_MONITORENTER",
105 justification="Synchronization on tasks in progress counter is intended")
106 class AsyncProcess {
107 private static final Log LOG = LogFactory.getLog(AsyncProcess.class);
108 protected static final AtomicLong COUNTER = new AtomicLong();
109
110 public static final String PRIMARY_CALL_TIMEOUT_KEY = "hbase.client.primaryCallTimeout.multiget";
111
112
113
114
115
116
117
118 public static final String START_LOG_ERRORS_AFTER_COUNT_KEY =
119 "hbase.client.start.log.errors.counter";
120 public static final int DEFAULT_START_LOG_ERRORS_AFTER_COUNT = 9;
121
122
123
124
125 public static final String LOG_DETAILS_FOR_BATCH_ERROR = "hbase.client.log.batcherrors.details";
126
127 private final int thresholdToLogUndoneTaskDetails;
128 private static final String THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS =
129 "hbase.client.threshold.log.details";
130 private static final int DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS = 10;
131 private final int THRESHOLD_TO_LOG_REGION_DETAILS = 2;
132
133
134
135
136 public static final String HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = "hbase.client.max.perrequest.heapsize";
137
138
139
140
141 public static final long DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE = 4194304;
142
143
144
145
146 public static final String HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = "hbase.client.max.submit.heapsize";
147
148
149
150 public static final long DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE = DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE;
151
152
153
154
155
156
157
158 public static interface AsyncRequestFuture {
159 public boolean hasError();
160 public RetriesExhaustedWithDetailsException getErrors();
161 public List<? extends Row> getFailedOperations();
162 public Object[] getResults() throws InterruptedIOException;
163
164 public void waitUntilDone() throws InterruptedIOException;
165 }
166
167
168
169
170 private static final AsyncRequestFuture NO_REQS_RESULT = new AsyncRequestFuture() {
171
172 final Object[] result = new Object[0];
173
174 @Override
175 public boolean hasError() {
176 return false;
177 }
178
179 @Override
180 public RetriesExhaustedWithDetailsException getErrors() {
181 return null;
182 }
183
184 @Override
185 public List<? extends Row> getFailedOperations() {
186 return null;
187 }
188
189 @Override
190 public Object[] getResults() {
191 return result;
192 }
193
194 @Override
195 public void waitUntilDone() throws InterruptedIOException {
196 }
197 };
198
199
200
201
202
203 private static class ReplicaResultState {
204 public ReplicaResultState(int callCount) {
205 this.callCount = callCount;
206 }
207
208
209 int callCount;
210
211
212 BatchErrors replicaErrors = null;
213
214 @Override
215 public String toString() {
216 return "[call count " + callCount + "; errors " + replicaErrors + "]";
217 }
218 }
219
220
221
222 protected final long id;
223
224 protected final ClusterConnection connection;
225 protected final RpcRetryingCallerFactory rpcCallerFactory;
226 protected final RpcControllerFactory rpcFactory;
227 protected final BatchErrors globalErrors;
228 protected final ExecutorService pool;
229
230 protected final AtomicLong tasksInProgress = new AtomicLong(0);
231 protected final ConcurrentMap<byte[], AtomicInteger> taskCounterPerRegion =
232 new ConcurrentSkipListMap<byte[], AtomicInteger>(Bytes.BYTES_COMPARATOR);
233 protected final ConcurrentMap<ServerName, AtomicInteger> taskCounterPerServer =
234 new ConcurrentHashMap<ServerName, AtomicInteger>();
235
236 private final int startLogErrorsCnt;
237
238
239
240
241 protected final int maxTotalConcurrentTasks;
242
243
244
245
246 protected final long maxHeapSizePerRequest;
247 protected final long maxHeapSizeSubmit;
248
249
250
251
252
253
254 protected final int maxConcurrentTasksPerRegion;
255
256
257
258
259 protected final int maxConcurrentTasksPerServer;
260 protected final long pause;
261 protected final long pauseForCQTBE;
262 protected int numTries;
263 protected int serverTrackerTimeout;
264 protected int rpcTimeout;
265 protected int operationTimeout;
266 protected long primaryCallTimeoutMicroseconds;
267
268 private final boolean logBatchErrorDetails;
269
270
271 protected static class BatchErrors {
272 private final List<Throwable> throwables = new ArrayList<Throwable>();
273 private final List<Row> actions = new ArrayList<Row>();
274 private final List<String> addresses = new ArrayList<String>();
275
276 public synchronized void add(Throwable ex, Row row, ServerName serverName) {
277 if (row == null){
278 throw new IllegalArgumentException("row cannot be null. location=" + serverName);
279 }
280
281 throwables.add(ex);
282 actions.add(row);
283 addresses.add(serverName != null ? serverName.toString() : "null");
284 }
285
286 public boolean hasErrors() {
287 return !throwables.isEmpty();
288 }
289
290 private synchronized RetriesExhaustedWithDetailsException makeException(boolean logDetails) {
291 if (logDetails) {
292 LOG.error("Exception occurred! Exception details: " + throwables + ";\nActions: "
293 + actions);
294 }
295 return new RetriesExhaustedWithDetailsException(new ArrayList<Throwable>(throwables),
296 new ArrayList<Row>(actions), new ArrayList<String>(addresses));
297 }
298
299 public synchronized void clear() {
300 throwables.clear();
301 actions.clear();
302 addresses.clear();
303 }
304
305 public synchronized void merge(BatchErrors other) {
306 throwables.addAll(other.throwables);
307 actions.addAll(other.actions);
308 addresses.addAll(other.addresses);
309 }
310 }
311 public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService pool,
312 RpcRetryingCallerFactory rpcCaller, boolean useGlobalErrors, RpcControllerFactory rpcFactory,
313 int rpcTimeout) {
314 if (hc == null) {
315 throw new IllegalArgumentException("HConnection cannot be null.");
316 }
317
318 this.connection = hc;
319 this.pool = pool;
320 this.globalErrors = useGlobalErrors ? new BatchErrors() : null;
321
322 this.id = COUNTER.incrementAndGet();
323
324 ConnectionConfiguration connConf =
325 hc.getConfiguration() == conf
326 ? hc.getConnectionConfiguration()
327
328 : new ConnectionConfiguration(conf);
329 if (connConf == null) {
330
331 connConf = new ConnectionConfiguration(conf);
332 }
333
334 this.pause = connConf.getPause();
335 this.pauseForCQTBE = connConf.getPauseForCQTBE();
336
337 this.numTries = connConf.getRetriesNumber();
338 this.rpcTimeout = rpcTimeout;
339 this.operationTimeout = connConf.getOperationTimeout();
340
341
342
343 AsyncProcess globalAsyncProcess = hc.getConfiguration() == conf ? hc.getAsyncProcess() : null;
344
345 this.primaryCallTimeoutMicroseconds =
346 globalAsyncProcess == null
347 ? conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000)
348 : globalAsyncProcess.primaryCallTimeoutMicroseconds;
349
350 this.maxTotalConcurrentTasks =
351 globalAsyncProcess == null
352 ? conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
353 HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)
354 : globalAsyncProcess.maxTotalConcurrentTasks;
355 this.maxConcurrentTasksPerServer =
356 globalAsyncProcess == null
357 ? conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
358 HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS)
359 : globalAsyncProcess.maxConcurrentTasksPerServer;
360 this.maxConcurrentTasksPerRegion =
361 globalAsyncProcess == null
362 ? conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
363 HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS)
364 : globalAsyncProcess.maxConcurrentTasksPerRegion;
365 this.maxHeapSizePerRequest =
366 globalAsyncProcess == null
367 ? conf.getLong(HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
368 DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE)
369 : globalAsyncProcess.maxHeapSizePerRequest;
370 this.maxHeapSizeSubmit =
371 globalAsyncProcess == null
372 ? conf.getLong(HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE)
373 : globalAsyncProcess.maxHeapSizeSubmit;
374 this.startLogErrorsCnt =
375 globalAsyncProcess == null
376 ? conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT)
377 : globalAsyncProcess.startLogErrorsCnt;
378
379 if (this.maxTotalConcurrentTasks <= 0) {
380 throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
381 }
382 if (this.maxConcurrentTasksPerServer <= 0) {
383 throw new IllegalArgumentException("maxConcurrentTasksPerServer=" +
384 maxConcurrentTasksPerServer);
385 }
386 if (this.maxConcurrentTasksPerRegion <= 0) {
387 throw new IllegalArgumentException("maxConcurrentTasksPerRegion=" +
388 maxConcurrentTasksPerRegion);
389 }
390 if (this.maxHeapSizePerRequest <= 0) {
391 throw new IllegalArgumentException("maxHeapSizePerServer=" +
392 maxHeapSizePerRequest);
393 }
394
395 if (this.maxHeapSizeSubmit <= 0) {
396 throw new IllegalArgumentException("maxHeapSizeSubmit=" +
397 maxHeapSizeSubmit);
398 }
399
400
401
402
403
404
405
406 this.serverTrackerTimeout = 0;
407 for (int i = 0; i < this.numTries; ++i) {
408 serverTrackerTimeout = (int) (serverTrackerTimeout + ConnectionUtils.getPauseTime(this.pause, i));
409 }
410
411 this.rpcCallerFactory = rpcCaller;
412 this.rpcFactory = rpcFactory;
413 this.logBatchErrorDetails =
414 globalAsyncProcess == null
415 ? conf.getBoolean(LOG_DETAILS_FOR_BATCH_ERROR, false)
416 : globalAsyncProcess.logBatchErrorDetails;
417
418 this.thresholdToLogUndoneTaskDetails =
419 globalAsyncProcess == null
420 ? conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS,
421 DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS)
422 : globalAsyncProcess.thresholdToLogUndoneTaskDetails;
423 }
424
425 public void setRpcTimeout(int rpcTimeout) {
426 this.rpcTimeout = rpcTimeout;
427 }
428
429 public void setOperationTimeout(int operationTimeout) {
430 this.operationTimeout = operationTimeout;
431 }
432
433
434
435
436
437 ExecutorService getPool(ExecutorService pool) {
438 if (pool != null) {
439 return pool;
440 }
441 if (this.pool != null) {
442 return this.pool;
443 }
444 throw new RuntimeException("Neither AsyncProcess nor request have ExecutorService");
445 }
446
447
448
449
450
451 public <CResult> AsyncRequestFuture submit(TableName tableName, final List<? extends Row> rows,
452 boolean atLeastOne, Batch.Callback<CResult> callback, boolean needResults)
453 throws InterruptedIOException {
454 return submit(null, tableName, rows, atLeastOne, callback, needResults);
455 }
456
457
458
459
460 public <CResult> AsyncRequestFuture submit(TableName tableName,
461 final RowAccess<? extends Row> rows, boolean atLeastOne, Batch.Callback<CResult> callback,
462 boolean needResults) throws InterruptedIOException {
463 return submit(null, tableName, rows, atLeastOne, callback, needResults);
464 }
465
466
467
468
469 public <CResult> AsyncRequestFuture submit(ExecutorService pool, TableName tableName,
470 List<? extends Row> rows, boolean atLeastOne, Batch.Callback<CResult> callback,
471 boolean needResults) throws InterruptedIOException {
472 return submit(pool, tableName, new ListRowAccess(rows), atLeastOne,
473 callback, needResults);
474 }
475
476
477
478
479
480
481
482
483
484
485
486
487
488 public <CResult> AsyncRequestFuture submit(ExecutorService pool, TableName tableName,
489 RowAccess<? extends Row> rows, boolean atLeastOne, Batch.Callback<CResult> callback,
490 boolean needResults) throws InterruptedIOException {
491 if (rows.isEmpty()) {
492 return NO_REQS_RESULT;
493 }
494
495 Map<ServerName, MultiAction<Row>> actionsByServer =
496 new HashMap<ServerName, MultiAction<Row>>();
497 List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(rows.size());
498
499 NonceGenerator ng = this.connection.getNonceGenerator();
500 long nonceGroup = ng.getNonceGroup();
501
502
503 List<Exception> locationErrors = null;
504 List<Integer> locationErrorRows = null;
505 RowCheckerHost checker = createRowCheckerHost();
506 boolean firstIter = true;
507 do {
508
509 waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1, tableName.getNameAsString());
510 int posInList = -1;
511 if (!firstIter) {
512 checker.reset();
513 }
514 Iterator<? extends Row> it = rows.iterator();
515 while (it.hasNext()) {
516 Row r = it.next();
517 HRegionLocation loc;
518 try {
519 if (r == null) {
520 throw new IllegalArgumentException("#" + id + ", row cannot be null");
521 }
522
523 RegionLocations locs = connection.locateRegion(
524 tableName, r.getRow(), true, true, RegionReplicaUtil.DEFAULT_REPLICA_ID);
525 if (locs == null || locs.isEmpty() || locs.getDefaultRegionLocation() == null) {
526 throw new IOException("#" + id + ", no location found, aborting submit for"
527 + " tableName=" + tableName + " rowkey=" + Bytes.toStringBinary(r.getRow()));
528 }
529 loc = locs.getDefaultRegionLocation();
530 } catch (IOException ex) {
531 locationErrors = new ArrayList<Exception>();
532 locationErrorRows = new ArrayList<Integer>();
533 LOG.error("Failed to get region location ", ex);
534
535
536 int priority = HConstants.NORMAL_QOS;
537 if (r instanceof Mutation) {
538 priority = ((Mutation) r).getPriority();
539 }
540 retainedActions.add(new Action<Row>(r, ++posInList, priority));
541 locationErrors.add(ex);
542 locationErrorRows.add(posInList);
543 it.remove();
544 break;
545 }
546 long rowSize = (r instanceof Mutation) ? ((Mutation) r).heapSize() : 0;
547 ReturnCode code = checker.canTakeOperation(loc, rowSize);
548 if (code == ReturnCode.END) {
549 break;
550 }
551 if (code == ReturnCode.INCLUDE) {
552 int priority = HConstants.NORMAL_QOS;
553 if (r instanceof Mutation) {
554 priority = ((Mutation) r).getPriority();
555 }
556 Action<Row> action = new Action<Row>(r, ++posInList, priority);
557 setNonce(ng, r, action);
558 retainedActions.add(action);
559
560 byte[] regionName = loc.getRegionInfo().getRegionName();
561 addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
562 it.remove();
563 }
564 }
565 firstIter = false;
566 } while (retainedActions.isEmpty() && atLeastOne && (locationErrors == null));
567
568 if (retainedActions.isEmpty()) return NO_REQS_RESULT;
569
570 return submitMultiActions(tableName, retainedActions, nonceGroup, callback, null, needResults,
571 locationErrors, locationErrorRows, actionsByServer, pool);
572 }
573
574 private RowCheckerHost createRowCheckerHost() {
575 return new RowCheckerHost(Arrays.asList(
576 new TaskCountChecker(maxTotalConcurrentTasks,
577 maxConcurrentTasksPerServer,
578 maxConcurrentTasksPerRegion,
579 tasksInProgress,
580 taskCounterPerServer,
581 taskCounterPerRegion)
582 , new RequestSizeChecker(maxHeapSizePerRequest)
583 , new SubmittedSizeChecker(maxHeapSizeSubmit)
584 ));
585 }
586 <CResult> AsyncRequestFuture submitMultiActions(TableName tableName,
587 List<Action<Row>> retainedActions, long nonceGroup, Batch.Callback<CResult> callback,
588 Object[] results, boolean needResults, List<Exception> locationErrors,
589 List<Integer> locationErrorRows, Map<ServerName, MultiAction<Row>> actionsByServer,
590 ExecutorService pool) {
591 AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(
592 tableName, retainedActions, nonceGroup, pool, callback, results, needResults, null,
593 operationTimeout, rpcTimeout);
594
595 if (locationErrors != null) {
596 for (int i = 0; i < locationErrors.size(); ++i) {
597 int originalIndex = locationErrorRows.get(i);
598 Row row = retainedActions.get(originalIndex).getAction();
599 ars.manageError(originalIndex, row,
600 Retry.NO_LOCATION_PROBLEM, locationErrors.get(i), null);
601 }
602 }
603 ars.sendMultiAction(actionsByServer, 1, null, false);
604 return ars;
605 }
606
607
608
609
610
611
612
613
614
615 private static void addAction(ServerName server, byte[] regionName, Action<Row> action,
616 Map<ServerName, MultiAction<Row>> actionsByServer, long nonceGroup) {
617 MultiAction<Row> multiAction = actionsByServer.get(server);
618 if (multiAction == null) {
619 multiAction = new MultiAction<Row>();
620 actionsByServer.put(server, multiAction);
621 }
622 if (action.hasNonce() && !multiAction.hasNonceGroup()) {
623 multiAction.setNonceGroup(nonceGroup);
624 }
625
626 multiAction.add(regionName, action);
627 }
628
629
630
631
632 public <CResult> AsyncRequestFuture submitAll(TableName tableName,
633 List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results) {
634 return submitAll(null, tableName, rows, callback, results, null, operationTimeout, rpcTimeout);
635 }
636
637 public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
638 List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results) {
639 return submitAll(pool, tableName, rows, callback, results, null, operationTimeout, rpcTimeout);
640 }
641
642
643
644
645
646
647
648
649
650
651 public <CResult> AsyncRequestFuture submitAll(ExecutorService pool, TableName tableName,
652 List<? extends Row> rows, Batch.Callback<CResult> callback, Object[] results,
653 PayloadCarryingServerCallable callable, int operationTimeout, int rpcTimeout) {
654 List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
655
656
657 int posInList = -1;
658 NonceGenerator ng = this.connection.getNonceGenerator();
659 int highestPriority = HConstants.PRIORITY_UNSET;
660 for (Row r : rows) {
661 posInList++;
662 if (r instanceof Put) {
663 Put put = (Put) r;
664 if (put.isEmpty()) {
665 throw new IllegalArgumentException("No columns to insert for #" + (posInList+1)+ " item");
666 }
667 highestPriority = Math.max(put.getPriority(), highestPriority);
668 }
669 Action<Row> action = new Action<Row>(r, posInList, highestPriority);
670 setNonce(ng, r, action);
671 actions.add(action);
672 }
673 AsyncRequestFutureImpl<CResult> ars = createAsyncRequestFuture(
674 tableName, actions, ng.getNonceGroup(), getPool(pool), callback, results, results != null,
675 callable, operationTimeout, rpcTimeout);
676 ars.groupAndSendMultiAction(actions, 1);
677 return ars;
678 }
679
680 private static void setNonce(NonceGenerator ng, Row r, Action<Row> action) {
681 if (!(r instanceof Append) && !(r instanceof Increment)) return;
682 action.setNonce(ng.newNonce());
683 }
684
685
686
687
688
689
690
691
692
693
694 protected class AsyncRequestFutureImpl<CResult> implements AsyncRequestFuture {
695
696
697
698
699
700
701
702
703 private final class ReplicaCallIssuingRunnable implements Runnable {
704 private final long startTime;
705 private final List<Action<Row>> initialActions;
706
707 public ReplicaCallIssuingRunnable(List<Action<Row>> initialActions, long startTime) {
708 this.initialActions = initialActions;
709 this.startTime = startTime;
710 }
711
712 @Override
713 public void run() {
714 boolean done = false;
715 if (primaryCallTimeoutMicroseconds > 0) {
716 try {
717 done = waitUntilDone(startTime * 1000L + primaryCallTimeoutMicroseconds);
718 } catch (InterruptedException ex) {
719 LOG.error("Replica thread was interrupted - no replica calls: " + ex.getMessage());
720 return;
721 }
722 }
723 if (done) return;
724 Map<ServerName, MultiAction<Row>> actionsByServer =
725 new HashMap<ServerName, MultiAction<Row>>();
726 List<Action<Row>> unknownLocActions = new ArrayList<Action<Row>>();
727 if (replicaGetIndices == null) {
728 for (int i = 0; i < results.length; ++i) {
729 addReplicaActions(i, actionsByServer, unknownLocActions);
730 }
731 } else {
732 for (int replicaGetIndice : replicaGetIndices) {
733 addReplicaActions(replicaGetIndice, actionsByServer, unknownLocActions);
734 }
735 }
736 if (!actionsByServer.isEmpty()) {
737 sendMultiAction(actionsByServer, 1, null, unknownLocActions.isEmpty());
738 }
739 if (!unknownLocActions.isEmpty()) {
740 actionsByServer = new HashMap<ServerName, MultiAction<Row>>();
741 for (Action<Row> action : unknownLocActions) {
742 addReplicaActionsAgain(action, actionsByServer);
743 }
744
745 if (!actionsByServer.isEmpty()) {
746 sendMultiAction(actionsByServer, 1, null, true);
747 }
748 }
749 }
750
751
752
753
754
755
756 private void addReplicaActions(int index, Map<ServerName, MultiAction<Row>> actionsByServer,
757 List<Action<Row>> unknownReplicaActions) {
758 if (results[index] != null) return;
759 Action<Row> action = initialActions.get(index);
760 RegionLocations loc = findAllLocationsOrFail(action, true);
761 if (loc == null) return;
762 HRegionLocation[] locs = loc.getRegionLocations();
763 if (locs.length == 1) {
764 LOG.warn("No replicas found for " + action.getAction());
765 return;
766 }
767 synchronized (replicaResultLock) {
768
769
770
771 if (results[index] != null) return;
772
773
774 results[index] = new ReplicaResultState(locs.length);
775 }
776 for (int i = 1; i < locs.length; ++i) {
777 Action<Row> replicaAction = new Action<Row>(action, i);
778 if (locs[i] != null) {
779 addAction(locs[i].getServerName(), locs[i].getRegionInfo().getRegionName(),
780 replicaAction, actionsByServer, nonceGroup);
781 } else {
782 unknownReplicaActions.add(replicaAction);
783 }
784 }
785 }
786
787 private void addReplicaActionsAgain(
788 Action<Row> action, Map<ServerName, MultiAction<Row>> actionsByServer) {
789 if (action.getReplicaId() == RegionReplicaUtil.DEFAULT_REPLICA_ID) {
790 throw new AssertionError("Cannot have default replica here");
791 }
792 HRegionLocation loc = getReplicaLocationOrFail(action);
793 if (loc == null) return;
794 addAction(loc.getServerName(), loc.getRegionInfo().getRegionName(),
795 action, actionsByServer, nonceGroup);
796 }
797 }
798
799
800
801
802
803 class SingleServerRequestRunnable implements Runnable {
804 private final MultiAction<Row> multiAction;
805 private final int numAttempt;
806 private final ServerName server;
807 private final Set<PayloadCarryingServerCallable> callsInProgress;
808 SingleServerRequestRunnable(
809 MultiAction<Row> multiAction, int numAttempt, ServerName server,
810 Set<PayloadCarryingServerCallable> callsInProgress) {
811 this.multiAction = multiAction;
812 this.numAttempt = numAttempt;
813 this.server = server;
814 this.callsInProgress = callsInProgress;
815 }
816
817 @Override
818 public void run() {
819 MultiResponse res = null;
820 PayloadCarryingServerCallable callable = currentCallable;
821 try {
822
823 if (callable == null) {
824 callable = createCallable(server, tableName, multiAction);
825 }
826 RpcRetryingCaller<MultiResponse> caller = createCaller(callable, rpcTimeout);
827 try {
828 if (callsInProgress != null) {
829 callsInProgress.add(callable);
830 }
831 res = caller.callWithoutRetries(callable, operationTimeout);
832 if (res == null) {
833
834 return;
835 }
836 } catch (IOException e) {
837
838
839 receiveGlobalFailure(multiAction, server, numAttempt, e);
840 return;
841 } catch (Throwable t) {
842
843 LOG.error("#" + id + ", Caught throwable while calling. This is unexpected." +
844 " Retrying. Server is " + server + ", tableName=" + tableName, t);
845 receiveGlobalFailure(multiAction, server, numAttempt, t);
846 return;
847 }
848
849
850 receiveMultiAction(multiAction, server, res, numAttempt);
851 } catch (Throwable t) {
852
853 LOG.error("Internal AsyncProcess #" + id + " error for "
854 + tableName + " processing for " + server, t);
855 throw new RuntimeException(t);
856 } finally {
857 decTaskCounters(multiAction.getRegions(), server);
858 if (callsInProgress != null && callable != null && res != null) {
859 callsInProgress.remove(callable);
860 }
861 }
862 }
863 }
864
865 private final Batch.Callback<CResult> callback;
866 private final BatchErrors errors;
867 private final ConnectionManager.ServerErrorTracker errorsByServer;
868 private final ExecutorService pool;
869 private final Set<PayloadCarryingServerCallable> callsInProgress;
870
871
872 private final TableName tableName;
873 private final AtomicLong actionsInProgress = new AtomicLong(-1);
874
875
876
877
878
879 private final Object replicaResultLock = new Object();
880
881
882
883
884
885
886
887
888
889 private final Object[] results;
890
891
892
893 private final int[] replicaGetIndices;
894 private final boolean hasAnyReplicaGets;
895 private final long nonceGroup;
896 private PayloadCarryingServerCallable currentCallable;
897 private int operationTimeout;
898 private int rpcTimeout;
899 private RetryingTimeTracker tracker;
900
901 public AsyncRequestFutureImpl(TableName tableName, List<Action<Row>> actions, long nonceGroup,
902 ExecutorService pool, boolean needResults, Object[] results,
903 Batch.Callback<CResult> callback, PayloadCarryingServerCallable callable,
904 int operationTimeout, int rpcTimeout) {
905 this.pool = pool;
906 this.callback = callback;
907 this.nonceGroup = nonceGroup;
908 this.tableName = tableName;
909 this.actionsInProgress.set(actions.size());
910 if (results != null) {
911 assert needResults;
912 if (results.length != actions.size()) {
913 throw new AssertionError("results.length");
914 }
915 this.results = results;
916 for (int i = 0; i != this.results.length; ++i) {
917 results[i] = null;
918 }
919 } else {
920 this.results = needResults ? new Object[actions.size()] : null;
921 }
922 List<Integer> replicaGetIndices = null;
923 boolean hasAnyReplicaGets = false;
924 if (needResults) {
925
926
927
928
929
930 boolean hasAnyNonReplicaReqs = false;
931 int posInList = 0;
932 for (Action<Row> action : actions) {
933 boolean isReplicaGet = isReplicaGet(action.getAction());
934 if (isReplicaGet) {
935 hasAnyReplicaGets = true;
936 if (hasAnyNonReplicaReqs) {
937 if (replicaGetIndices == null) {
938 replicaGetIndices = new ArrayList<Integer>(actions.size() - 1);
939 }
940 replicaGetIndices.add(posInList);
941 }
942 } else if (!hasAnyNonReplicaReqs) {
943
944 hasAnyNonReplicaReqs = true;
945 if (posInList > 0) {
946
947
948 replicaGetIndices = new ArrayList<Integer>(actions.size() - 1);
949 for (int i = 0; i < posInList; ++i) {
950 replicaGetIndices.add(i);
951 }
952 }
953 }
954 ++posInList;
955 }
956 }
957 this.hasAnyReplicaGets = hasAnyReplicaGets;
958 if (replicaGetIndices != null) {
959 this.replicaGetIndices = new int[replicaGetIndices.size()];
960 int i = 0;
961 for (Integer el : replicaGetIndices) {
962 this.replicaGetIndices[i++] = el;
963 }
964 } else {
965 this.replicaGetIndices = null;
966 }
967 this.callsInProgress = !hasAnyReplicaGets ? null :
968 Collections.newSetFromMap(
969 new ConcurrentHashMap<PayloadCarryingServerCallable, Boolean>());
970
971 this.errorsByServer = createServerErrorTracker();
972 this.errors = (globalErrors != null) ? globalErrors : new BatchErrors();
973 this.currentCallable = callable;
974 this.operationTimeout = operationTimeout;
975 this.rpcTimeout = rpcTimeout;
976 if (callable == null) {
977 tracker = new RetryingTimeTracker();
978 tracker.start();
979 }
980 }
981
982 public Set<PayloadCarryingServerCallable> getCallsInProgress() {
983 return callsInProgress;
984 }
985
986 SingleServerRequestRunnable createSingleServerRequest(MultiAction<Row> multiAction, int numAttempt, ServerName server,
987 Set<PayloadCarryingServerCallable> callsInProgress) {
988 return new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress);
989 }
990
991
992
993
994
995
996
997 private void groupAndSendMultiAction(List<Action<Row>> currentActions, int numAttempt) {
998 Map<ServerName, MultiAction<Row>> actionsByServer =
999 new HashMap<ServerName, MultiAction<Row>>();
1000
1001 boolean isReplica = false;
1002 List<Action<Row>> unknownReplicaActions = null;
1003 for (Action<Row> action : currentActions) {
1004 RegionLocations locs = findAllLocationsOrFail(action, true);
1005 if (locs == null) continue;
1006 boolean isReplicaAction = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
1007 if (isReplica && !isReplicaAction) {
1008
1009 throw new AssertionError("Replica and non-replica actions in the same retry");
1010 }
1011 isReplica = isReplicaAction;
1012 HRegionLocation loc = locs.getRegionLocation(action.getReplicaId());
1013 if (loc == null || loc.getServerName() == null) {
1014 if (isReplica) {
1015 if (unknownReplicaActions == null) {
1016 unknownReplicaActions = new ArrayList<Action<Row>>();
1017 }
1018 unknownReplicaActions.add(action);
1019 } else {
1020
1021 manageLocationError(action, null);
1022 }
1023 } else {
1024 byte[] regionName = loc.getRegionInfo().getRegionName();
1025 addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
1026 }
1027 }
1028 boolean doStartReplica = (numAttempt == 1 && !isReplica && hasAnyReplicaGets);
1029 boolean hasUnknown = unknownReplicaActions != null && !unknownReplicaActions.isEmpty();
1030
1031 if (!actionsByServer.isEmpty()) {
1032
1033 sendMultiAction(actionsByServer, numAttempt, (doStartReplica && !hasUnknown)
1034 ? currentActions : null, numAttempt > 1 && !hasUnknown);
1035 }
1036
1037 if (hasUnknown) {
1038 actionsByServer = new HashMap<ServerName, MultiAction<Row>>();
1039 for (Action<Row> action : unknownReplicaActions) {
1040 HRegionLocation loc = getReplicaLocationOrFail(action);
1041 if (loc == null) continue;
1042 byte[] regionName = loc.getRegionInfo().getRegionName();
1043 addAction(loc.getServerName(), regionName, action, actionsByServer, nonceGroup);
1044 }
1045 if (!actionsByServer.isEmpty()) {
1046 sendMultiAction(
1047 actionsByServer, numAttempt, doStartReplica ? currentActions : null, true);
1048 }
1049 }
1050 }
1051
1052 private HRegionLocation getReplicaLocationOrFail(Action<Row> action) {
1053
1054
1055 int replicaId = action.getReplicaId();
1056 RegionLocations locs = findAllLocationsOrFail(action, true);
1057 if (locs == null) return null;
1058 HRegionLocation loc = locs.getRegionLocation(replicaId);
1059 if (loc == null || loc.getServerName() == null) {
1060 locs = findAllLocationsOrFail(action, false);
1061 if (locs == null) return null;
1062 loc = locs.getRegionLocation(replicaId);
1063 }
1064 if (loc == null || loc.getServerName() == null) {
1065 manageLocationError(action, null);
1066 return null;
1067 }
1068 return loc;
1069 }
1070
1071 private void manageLocationError(Action<Row> action, Exception ex) {
1072 String msg = "Cannot get replica " + action.getReplicaId()
1073 + " location for " + action.getAction();
1074 LOG.error(msg);
1075 if (ex == null) {
1076 ex = new IOException(msg);
1077 }
1078 manageError(action.getOriginalIndex(), action.getAction(),
1079 Retry.NO_LOCATION_PROBLEM, ex, null);
1080 }
1081
1082 private RegionLocations findAllLocationsOrFail(Action<Row> action, boolean useCache) {
1083 if (action.getAction() == null) throw new IllegalArgumentException("#" + id +
1084 ", row cannot be null");
1085 RegionLocations loc = null;
1086 try {
1087 loc = connection.locateRegion(
1088 tableName, action.getAction().getRow(), useCache, true, action.getReplicaId());
1089 } catch (IOException ex) {
1090 manageLocationError(action, ex);
1091 }
1092 return loc;
1093 }
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103 private void sendMultiAction(Map<ServerName, MultiAction<Row>> actionsByServer,
1104 int numAttempt, List<Action<Row>> actionsForReplicaThread, boolean reuseThread) {
1105
1106
1107 int actionsRemaining = actionsByServer.size();
1108
1109 for (Map.Entry<ServerName, MultiAction<Row>> e : actionsByServer.entrySet()) {
1110 ServerName server = e.getKey();
1111 MultiAction<Row> multiAction = e.getValue();
1112 Collection<? extends Runnable> runnables = getNewMultiActionRunnable(server, multiAction,
1113 numAttempt);
1114
1115
1116 if (runnables.size() > actionsRemaining) {
1117 actionsRemaining = runnables.size();
1118 }
1119
1120
1121
1122
1123 for (Runnable runnable : runnables) {
1124 if ((--actionsRemaining == 0) && reuseThread
1125 && numAttempt % HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER != 0) {
1126 runnable.run();
1127 } else {
1128 try {
1129 pool.submit(runnable);
1130 } catch (Throwable t) {
1131 if (t instanceof RejectedExecutionException) {
1132
1133
1134 LOG.warn("#" + id + ", the task was rejected by the pool. This is unexpected." +
1135 " Server is " + server.getServerName(), t);
1136 } else {
1137
1138 LOG.warn("Caught unexpected exception/error: ", t);
1139 }
1140 decTaskCounters(multiAction.getRegions(), server);
1141
1142
1143 receiveGlobalFailure(multiAction, server, numAttempt, t);
1144 }
1145 }
1146 }
1147 }
1148
1149 if (actionsForReplicaThread != null) {
1150 startWaitingForReplicaCalls(actionsForReplicaThread);
1151 }
1152 }
1153
1154 private Collection<? extends Runnable> getNewMultiActionRunnable(ServerName server,
1155 MultiAction<Row> multiAction,
1156 int numAttempt) {
1157
1158 if (AsyncProcess.this.connection.getStatisticsTracker() == null) {
1159 if (connection.getConnectionMetrics() != null) {
1160 connection.getConnectionMetrics().incrNormalRunners();
1161 }
1162 incTaskCounters(multiAction.getRegions(), server);
1163 SingleServerRequestRunnable runnable = createSingleServerRequest(multiAction, numAttempt, server, callsInProgress);
1164 return Collections.singletonList(Trace.wrap("AsyncProcess.sendMultiAction", runnable));
1165 }
1166
1167
1168 Map<Long, DelayingRunner> actions = new HashMap<Long, DelayingRunner>(multiAction
1169 .size());
1170
1171
1172 for (Map.Entry<byte[], List<Action<Row>>> e : multiAction.actions.entrySet()) {
1173 Long backoff = getBackoff(server, e.getKey());
1174 DelayingRunner runner = actions.get(backoff);
1175 if (runner == null) {
1176 actions.put(backoff, new DelayingRunner(backoff, e));
1177 } else {
1178 runner.add(e);
1179 }
1180 }
1181
1182 List<Runnable> toReturn = new ArrayList<Runnable>(actions.size());
1183 for (DelayingRunner runner : actions.values()) {
1184 incTaskCounters(runner.getActions().getRegions(), server);
1185 String traceText = "AsyncProcess.sendMultiAction";
1186 Runnable runnable = createSingleServerRequest(runner.getActions(), numAttempt, server, callsInProgress);
1187
1188 if (runner.getSleepTime() > 0) {
1189 runner.setRunner(runnable);
1190 traceText = "AsyncProcess.clientBackoff.sendMultiAction";
1191 runnable = runner;
1192 if (connection.getConnectionMetrics() != null) {
1193 connection.getConnectionMetrics().incrDelayRunners();
1194 connection.getConnectionMetrics().updateDelayInterval(runner.getSleepTime());
1195 }
1196 } else {
1197 if (connection.getConnectionMetrics() != null) {
1198 connection.getConnectionMetrics().incrNormalRunners();
1199 }
1200 }
1201 runnable = Trace.wrap(traceText, runnable);
1202 toReturn.add(runnable);
1203
1204 }
1205 return toReturn;
1206 }
1207
1208
1209
1210
1211
1212
1213
1214 private Long getBackoff(ServerName server, byte[] regionName) {
1215 ServerStatisticTracker tracker = AsyncProcess.this.connection.getStatisticsTracker();
1216 ServerStatistics stats = tracker.getStats(server);
1217 return AsyncProcess.this.connection.getBackoffPolicy()
1218 .getBackoffTime(server, regionName, stats);
1219 }
1220
1221
1222
1223
1224 private void startWaitingForReplicaCalls(List<Action<Row>> actionsForReplicaThread) {
1225 long startTime = EnvironmentEdgeManager.currentTime();
1226 ReplicaCallIssuingRunnable replicaRunnable = new ReplicaCallIssuingRunnable(
1227 actionsForReplicaThread, startTime);
1228 if (primaryCallTimeoutMicroseconds == 0) {
1229
1230 replicaRunnable.run();
1231 } else {
1232
1233
1234 try {
1235 pool.submit(replicaRunnable);
1236 } catch (RejectedExecutionException ree) {
1237 LOG.warn("#" + id + ", replica task was rejected by the pool - no replica calls", ree);
1238 }
1239 }
1240 }
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252 public Retry manageError(int originalIndex, Row row, Retry canRetry,
1253 Throwable throwable, ServerName server) {
1254 if (canRetry == Retry.YES
1255 && throwable != null && (throwable instanceof DoNotRetryIOException ||
1256 throwable instanceof NeedUnmanagedConnectionException)) {
1257 canRetry = Retry.NO_NOT_RETRIABLE;
1258 }
1259
1260 if (canRetry != Retry.YES) {
1261
1262 setError(originalIndex, row, throwable, server);
1263 } else if (isActionComplete(originalIndex, row)) {
1264 canRetry = Retry.NO_OTHER_SUCCEEDED;
1265 }
1266 return canRetry;
1267 }
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277 private void receiveGlobalFailure(
1278 MultiAction<Row> rsActions, ServerName server, int numAttempt, Throwable t) {
1279 errorsByServer.reportServerError(server);
1280 Retry canRetry = errorsByServer.canRetryMore(numAttempt)
1281 ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED;
1282
1283 if (tableName == null && ClientExceptionsUtil.isMetaClearingException(t)) {
1284
1285 connection.clearCaches(server);
1286 }
1287 int failed = 0, stopped = 0;
1288 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
1289 for (Map.Entry<byte[], List<Action<Row>>> e : rsActions.actions.entrySet()) {
1290 byte[] regionName = e.getKey();
1291 byte[] row = e.getValue().iterator().next().getAction().getRow();
1292
1293
1294 try {
1295 if (tableName != null) {
1296 connection.updateCachedLocations(tableName, regionName, row,
1297 ClientExceptionsUtil.isMetaClearingException(t) ? null : t, server);
1298 }
1299 } catch (Throwable ex) {
1300
1301
1302 LOG.error("Couldn't update cached region locations: " + ex);
1303 }
1304 for (Action<Row> action : e.getValue()) {
1305 Retry retry = manageError(
1306 action.getOriginalIndex(), action.getAction(), canRetry, t, server);
1307 if (retry == Retry.YES) {
1308 toReplay.add(action);
1309 } else if (retry == Retry.NO_OTHER_SUCCEEDED) {
1310 ++stopped;
1311 } else {
1312 ++failed;
1313 }
1314 }
1315 }
1316
1317 if (toReplay.isEmpty()) {
1318 logNoResubmit(server, numAttempt, rsActions.size(), t, failed, stopped);
1319 } else {
1320 resubmit(server, toReplay, numAttempt, rsActions.size(), t);
1321 }
1322 }
1323
1324
1325
1326
1327
1328 private void resubmit(ServerName oldServer, List<Action<Row>> toReplay,
1329 int numAttempt, int failureCount, Throwable throwable) {
1330
1331
1332
1333
1334
1335
1336
1337
1338 boolean retryImmediately = throwable instanceof RetryImmediatelyException;
1339 int nextAttemptNumber = retryImmediately ? numAttempt : numAttempt + 1;
1340 long backOffTime;
1341 if (retryImmediately) {
1342 backOffTime = 0;
1343 } else if (throwable instanceof CallQueueTooBigException) {
1344
1345 backOffTime = errorsByServer.calculateBackoffTime(oldServer, pauseForCQTBE);
1346 } else {
1347 backOffTime = errorsByServer.calculateBackoffTime(oldServer, pause);
1348 }
1349 if (numAttempt > startLogErrorsCnt) {
1350
1351
1352 LOG.info(createLog(numAttempt, failureCount, toReplay.size(),
1353 oldServer, throwable, backOffTime, true, null, -1, -1));
1354 }
1355
1356 try {
1357 if (backOffTime > 0) {
1358 Thread.sleep(backOffTime);
1359 }
1360 } catch (InterruptedException e) {
1361 LOG.warn("#" + id + ", not sent: " + toReplay.size() + " operations, " + oldServer, e);
1362 Thread.currentThread().interrupt();
1363 return;
1364 }
1365
1366 groupAndSendMultiAction(toReplay, nextAttemptNumber);
1367 }
1368
1369 private void logNoResubmit(ServerName oldServer, int numAttempt,
1370 int failureCount, Throwable throwable, int failed, int stopped) {
1371 if (failureCount != 0 || numAttempt > startLogErrorsCnt + 1) {
1372 String timeStr = new Date(errorsByServer.getStartTrackingTime()).toString();
1373 String logMessage = createLog(numAttempt, failureCount, 0, oldServer,
1374 throwable, -1, false, timeStr, failed, stopped);
1375 if (failed != 0) {
1376
1377 LOG.warn(logMessage);
1378 } else {
1379 LOG.info(logMessage);
1380 }
1381 }
1382 }
1383
1384 long getActionsInProgress() {
1385 return actionsInProgress.get();
1386 }
1387
1388
1389
1390
1391
1392
1393
1394
1395 private void receiveMultiAction(MultiAction<Row> multiAction,
1396 ServerName server, MultiResponse responses, int numAttempt) {
1397 assert responses != null;
1398
1399
1400
1401
1402
1403
1404
1405 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
1406 Throwable throwable = null;
1407 int failureCount = 0;
1408 Retry retry = null;
1409
1410 Map<byte[], MultiResponse.RegionResult> results = responses.getResults();
1411 updateStats(server, results);
1412
1413 int failed = 0;
1414 int stopped = 0;
1415
1416 for (Map.Entry<byte[], List<Action<Row>>> regionEntry : multiAction.actions.entrySet()) {
1417 byte[] regionName = regionEntry.getKey();
1418 Throwable regionException = responses.getExceptions().get(regionName);
1419 if (tableName == null && regionException != null &&
1420 ClientExceptionsUtil.isMetaClearingException(regionException)) {
1421
1422
1423
1424 connection.clearCaches(server);
1425 }
1426 Map<Integer, Object> regionResults;
1427 if (results.containsKey(regionName)) {
1428 regionResults = results.get(regionName).result;
1429 } else {
1430 regionResults = Collections.emptyMap();
1431 }
1432 boolean regionFailureRegistered = false;
1433 for (Action<Row> sentAction : regionEntry.getValue()) {
1434 Object result = regionResults.get(sentAction.getOriginalIndex());
1435 if (result == null) {
1436 if (regionException == null) {
1437 LOG.error("Server sent us neither results nor exceptions for " + Bytes
1438 .toStringBinary(regionName) + ", numAttempt:" + numAttempt);
1439 regionException = new RuntimeException("Invalid response");
1440 }
1441
1442
1443 result = regionException;
1444 }
1445
1446 if (result instanceof Throwable) {
1447 Row row = sentAction.getAction();
1448 throwable = regionException != null ? regionException
1449 : ClientExceptionsUtil.findException(result);
1450
1451 if (!regionFailureRegistered) {
1452 regionFailureRegistered = true;
1453 try {
1454 connection.updateCachedLocations(
1455 tableName, regionName, row.getRow(), result, server);
1456 } catch (Throwable ex) {
1457
1458
1459 LOG.error("Couldn't update cached region locations: " + ex);
1460 }
1461 }
1462 if (retry == null) {
1463 errorsByServer.reportServerError(server);
1464
1465 retry =
1466 errorsByServer.canRetryMore(numAttempt) ? Retry.YES : Retry.NO_RETRIES_EXHAUSTED;
1467 }
1468 ++failureCount;
1469 switch (manageError(sentAction.getOriginalIndex(), row, retry, (Throwable) result,
1470 server)) {
1471 case YES:
1472 toReplay.add(sentAction);
1473 break;
1474 case NO_OTHER_SUCCEEDED:
1475 ++stopped;
1476 break;
1477 default:
1478 ++failed;
1479 break;
1480 }
1481 } else {
1482 if (callback != null) {
1483 try {
1484
1485
1486 this.callback.update(regionName, sentAction.getAction().getRow(), (CResult) result);
1487 } catch (Throwable t) {
1488 LOG.error("User callback threw an exception for "
1489 + Bytes.toStringBinary(regionName) + ", ignoring", t);
1490 }
1491 }
1492 setResult(sentAction, result);
1493 }
1494 }
1495 }
1496 if (toReplay.isEmpty()) {
1497 logNoResubmit(server, numAttempt, failureCount, throwable, failed, stopped);
1498 } else {
1499 resubmit(server, toReplay, numAttempt, failureCount, throwable);
1500 }
1501 }
1502
1503 private String createLog(int numAttempt, int failureCount, int replaySize, ServerName sn,
1504 Throwable error, long backOffTime, boolean willRetry, String startTime,
1505 int failed, int stopped) {
1506 StringBuilder sb = new StringBuilder();
1507 sb.append("#").append(id).append(", table=").append(tableName).append(", ")
1508 .append("attempt=").append(numAttempt)
1509 .append("/").append(numTries).append(" ");
1510
1511 if (failureCount > 0 || error != null){
1512 sb.append("failed=").append(failureCount).append("ops").append(", last exception: ").
1513 append(error == null ? "null" : error);
1514 } else {
1515 sb.append("succeeded");
1516 }
1517
1518 sb.append(" on ").append(sn).append(", tracking started ").append(startTime);
1519
1520 if (willRetry) {
1521 sb.append(", retrying after=").append(backOffTime).append("ms").
1522 append(", replay=").append(replaySize).append("ops");
1523 } else if (failureCount > 0) {
1524 if (stopped > 0) {
1525 sb.append("; not retrying ").append(stopped).append(" due to success from other replica");
1526 }
1527 if (failed > 0) {
1528 sb.append("; not retrying ").append(failed).append(" - final failure");
1529 }
1530
1531 }
1532
1533 return sb.toString();
1534 }
1535
1536
1537
1538
1539
1540
1541 private void setResult(Action<Row> action, Object result) {
1542 if (result == null) {
1543 throw new RuntimeException("Result cannot be null");
1544 }
1545 ReplicaResultState state = null;
1546 boolean isStale = !RegionReplicaUtil.isDefaultReplica(action.getReplicaId());
1547 int index = action.getOriginalIndex();
1548 if (results == null) {
1549 decActionCounter(index);
1550 return;
1551 } else if ((state = trySetResultSimple(
1552 index, action.getAction(), false, result, null, isStale)) == null) {
1553 return;
1554 }
1555 assert state != null;
1556
1557
1558
1559
1560
1561 synchronized (state) {
1562 if (state.callCount == 0) {
1563 return;
1564 }
1565 state.callCount = 0;
1566 }
1567 synchronized (replicaResultLock) {
1568 if (results[index] != state) {
1569 throw new AssertionError("We set the callCount but someone else replaced the result");
1570 }
1571 results[index] = result;
1572 }
1573
1574 decActionCounter(index);
1575 }
1576
1577
1578
1579
1580
1581
1582
1583
1584 private void setError(int index, Row row, Throwable throwable, ServerName server) {
1585 ReplicaResultState state = null;
1586 if (results == null) {
1587
1588
1589
1590 errors.add(throwable, row, server);
1591 decActionCounter(index);
1592 return;
1593 } else if ((state = trySetResultSimple(
1594 index, row, true, throwable, server, false)) == null) {
1595 return;
1596 }
1597 assert state != null;
1598 BatchErrors target = null;
1599 boolean isActionDone = false;
1600 synchronized (state) {
1601 switch (state.callCount) {
1602 case 0: return;
1603 case 1: {
1604 target = errors;
1605 isActionDone = true;
1606 break;
1607 }
1608 default: {
1609 assert state.callCount > 1;
1610 if (state.replicaErrors == null) {
1611 state.replicaErrors = new BatchErrors();
1612 }
1613 target = state.replicaErrors;
1614 break;
1615 }
1616 }
1617 --state.callCount;
1618 }
1619 target.add(throwable, row, server);
1620 if (isActionDone) {
1621 if (state.replicaErrors != null) {
1622 errors.merge(state.replicaErrors);
1623 }
1624
1625 synchronized (replicaResultLock) {
1626 if (results[index] != state) {
1627 throw new AssertionError("We set the callCount but someone else replaced the result");
1628 }
1629 results[index] = throwable;
1630 }
1631 decActionCounter(index);
1632 }
1633 }
1634
1635
1636
1637
1638
1639
1640
1641
1642 private boolean isActionComplete(int index, Row row) {
1643 if (!isReplicaGet(row)) return false;
1644 Object resObj = results[index];
1645 return (resObj != null) && (!(resObj instanceof ReplicaResultState)
1646 || ((ReplicaResultState)resObj).callCount == 0);
1647 }
1648
1649
1650
1651
1652
1653 private ReplicaResultState trySetResultSimple(int index, Row row, boolean isError,
1654 Object result, ServerName server, boolean isFromReplica) {
1655 Object resObj = null;
1656 if (!isReplicaGet(row)) {
1657 if (isFromReplica) {
1658 throw new AssertionError("Unexpected stale result for " + row);
1659 }
1660 results[index] = result;
1661 } else {
1662 synchronized (replicaResultLock) {
1663 if ((resObj = results[index]) == null) {
1664 if (isFromReplica) {
1665 throw new AssertionError("Unexpected stale result for " + row);
1666 }
1667 results[index] = result;
1668 }
1669 }
1670 }
1671
1672 ReplicaResultState rrs =
1673 (resObj instanceof ReplicaResultState) ? (ReplicaResultState)resObj : null;
1674 if (rrs == null && isError) {
1675
1676 errors.add((Throwable)result, row, server);
1677 }
1678
1679 if (resObj == null) {
1680
1681 decActionCounter(index);
1682 return null;
1683 }
1684 return rrs;
1685 }
1686
1687 private void decActionCounter(int index) {
1688 long actionsRemaining = actionsInProgress.decrementAndGet();
1689 if (actionsRemaining < 0) {
1690 String error = buildDetailedErrorMsg("Incorrect actions in progress", index);
1691 throw new AssertionError(error);
1692 } else if (actionsRemaining == 0) {
1693 synchronized (actionsInProgress) {
1694 actionsInProgress.notifyAll();
1695 }
1696 }
1697 }
1698
1699 private String buildDetailedErrorMsg(String string, int index) {
1700 StringBuilder error = new StringBuilder(string);
1701 error.append("; called for ").
1702 append(index).
1703 append(", actionsInProgress ").
1704 append(actionsInProgress.get()).
1705 append("; replica gets: ");
1706 if (replicaGetIndices != null) {
1707 for (int i = 0; i < replicaGetIndices.length; ++i) {
1708 error.append(replicaGetIndices[i]).append(", ");
1709 }
1710 } else {
1711 error.append(hasAnyReplicaGets ? "all" : "none");
1712 }
1713 error.append("; results ");
1714 if (results != null) {
1715 for (int i = 0; i < results.length; ++i) {
1716 Object o = results[i];
1717 error.append(((o == null) ? "null" : o.toString())).append(", ");
1718 }
1719 }
1720 return error.toString();
1721 }
1722
1723 @Override
1724 public void waitUntilDone() throws InterruptedIOException {
1725 try {
1726 waitUntilDone(Long.MAX_VALUE);
1727 } catch (InterruptedException iex) {
1728 throw new InterruptedIOException(iex.getMessage());
1729 } finally {
1730 if (callsInProgress != null) {
1731 for (PayloadCarryingServerCallable clb : callsInProgress) {
1732 clb.cancel();
1733 }
1734 }
1735 }
1736 }
1737
1738 private boolean waitUntilDone(long cutoff) throws InterruptedException {
1739 boolean hasWait = cutoff != Long.MAX_VALUE;
1740 long lastLog = EnvironmentEdgeManager.currentTime();
1741 long currentInProgress;
1742 while (0 != (currentInProgress = actionsInProgress.get())) {
1743 long now = EnvironmentEdgeManager.currentTime();
1744 if (hasWait && (now * 1000L) > cutoff) {
1745 return false;
1746 }
1747 if (!hasWait) {
1748 if (now > lastLog + 10000) {
1749 lastLog = now;
1750 LOG.info("#" + id + ", waiting for " + currentInProgress
1751 + " actions to finish on table: " + tableName);
1752 if (currentInProgress <= thresholdToLogUndoneTaskDetails) {
1753 logDetailsOfUndoneTasks(currentInProgress);
1754 }
1755 }
1756 }
1757 synchronized (actionsInProgress) {
1758 if (actionsInProgress.get() == 0) break;
1759 if (!hasWait) {
1760 actionsInProgress.wait(10);
1761 } else {
1762 long waitMicroSecond = Math.min(100000L, (cutoff - now * 1000L));
1763 TimeUnit.MICROSECONDS.timedWait(actionsInProgress, waitMicroSecond);
1764 }
1765 }
1766 }
1767 return true;
1768 }
1769
1770 @Override
1771 public boolean hasError() {
1772 return errors.hasErrors();
1773 }
1774
1775 @Override
1776 public List<? extends Row> getFailedOperations() {
1777 return errors.actions;
1778 }
1779
1780 @Override
1781 public RetriesExhaustedWithDetailsException getErrors() {
1782 return errors.makeException(logBatchErrorDetails);
1783 }
1784
1785 @Override
1786 public Object[] getResults() throws InterruptedIOException {
1787 waitUntilDone();
1788 return results;
1789 }
1790
1791
1792
1793
1794 protected MultiServerCallable<Row> createCallable(final ServerName server,
1795 TableName tableName, final MultiAction<Row> multi) {
1796 return new MultiServerCallable<Row>(connection, tableName, server,
1797 AsyncProcess.this.rpcFactory, multi, rpcTimeout, tracker, multi.getPriority());
1798 }
1799 }
1800
1801 protected void updateStats(ServerName server, Map<byte[], MultiResponse.RegionResult> results) {
1802 boolean metrics = AsyncProcess.this.connection.getConnectionMetrics() != null;
1803 boolean stats = AsyncProcess.this.connection.getStatisticsTracker() != null;
1804 if (!stats && !metrics) {
1805 return;
1806 }
1807 for (Map.Entry<byte[], MultiResponse.RegionResult> regionStats : results.entrySet()) {
1808 byte[] regionName = regionStats.getKey();
1809 ClientProtos.RegionLoadStats stat = regionStats.getValue().getStat();
1810 ResultStatsUtil.updateStats(AsyncProcess.this.connection.getStatisticsTracker(), server,
1811 regionName, stat);
1812 ResultStatsUtil.updateStats(AsyncProcess.this.connection.getConnectionMetrics(),
1813 server, regionName, stat);
1814 }
1815 }
1816
1817 <CResult> AsyncRequestFutureImpl<CResult> createAsyncRequestFuture(
1818 TableName tableName, List<Action<Row>> actions, long nonceGroup, ExecutorService pool,
1819 Batch.Callback<CResult> callback, Object[] results, boolean needResults,
1820 PayloadCarryingServerCallable callable, int operationTimeout, int rpcTimeout) {
1821 return new AsyncRequestFutureImpl<CResult>(
1822 tableName, actions, nonceGroup, getPool(pool), needResults,
1823 results, callback, callable, operationTimeout, rpcTimeout);
1824 }
1825
1826
1827
1828
1829 protected RpcRetryingCaller<MultiResponse> createCaller(PayloadCarryingServerCallable callable,
1830 int rpcTimeout) {
1831 return rpcCallerFactory.<MultiResponse> newCaller(rpcTimeout);
1832 }
1833
1834
1835 void waitUntilDone() throws InterruptedIOException {
1836 waitForMaximumCurrentTasks(0, null);
1837 }
1838
1839
1840 private void waitForMaximumCurrentTasks(int max, String tableName)
1841 throws InterruptedIOException {
1842 waitForMaximumCurrentTasks(max, tasksInProgress, id, tableName);
1843 }
1844
1845
1846 void waitForMaximumCurrentTasks(int max, final AtomicLong tasksInProgress, final long id,
1847 String tableName) throws InterruptedIOException {
1848 long lastLog = EnvironmentEdgeManager.currentTime();
1849 long currentInProgress, oldInProgress = Long.MAX_VALUE;
1850 while ((currentInProgress = tasksInProgress.get()) > max) {
1851 if (oldInProgress != currentInProgress) {
1852 long now = EnvironmentEdgeManager.currentTime();
1853 if (now > lastLog + 10000) {
1854 lastLog = now;
1855 LOG.info("#" + id + ", waiting for some tasks to finish. Expected max="
1856 + max + ", tasksInProgress=" + currentInProgress +
1857 " hasError=" + hasError() + (tableName == null ? "" : ", tableName=" + tableName));
1858 if (currentInProgress <= thresholdToLogUndoneTaskDetails) {
1859 logDetailsOfUndoneTasks(currentInProgress);
1860 }
1861 }
1862 }
1863 oldInProgress = currentInProgress;
1864 try {
1865 synchronized (tasksInProgress) {
1866 if (tasksInProgress.get() == oldInProgress) {
1867 tasksInProgress.wait(10);
1868 }
1869 }
1870 } catch (InterruptedException e) {
1871 throw new InterruptedIOException("#" + id + ", interrupted." +
1872 " currentNumberOfTask=" + currentInProgress);
1873 }
1874 }
1875 }
1876
1877 private void logDetailsOfUndoneTasks(long taskInProgress) {
1878 ArrayList<ServerName> servers = new ArrayList<ServerName>();
1879 for (Map.Entry<ServerName, AtomicInteger> entry : taskCounterPerServer.entrySet()) {
1880 if (entry.getValue().get() > 0) {
1881 servers.add(entry.getKey());
1882 }
1883 }
1884 LOG.info("Left over " + taskInProgress + " task(s) are processed on server(s): " + servers);
1885 if (taskInProgress <= THRESHOLD_TO_LOG_REGION_DETAILS) {
1886 ArrayList<String> regions = new ArrayList<String>();
1887 for (Map.Entry<byte[], AtomicInteger> entry : taskCounterPerRegion.entrySet()) {
1888 if (entry.getValue().get() > 0) {
1889 regions.add(Bytes.toString(entry.getKey()));
1890 }
1891 }
1892 LOG.info("Regions against which left over task(s) are processed: " + regions);
1893 }
1894 }
1895
1896
1897
1898
1899
1900
1901 public boolean hasError() {
1902 return globalErrors.hasErrors();
1903 }
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915 public RetriesExhaustedWithDetailsException waitForAllPreviousOpsAndReset(
1916 List<Row> failedRows, String tableName) throws InterruptedIOException {
1917 waitForMaximumCurrentTasks(0, tableName);
1918 if (!globalErrors.hasErrors()) {
1919 return null;
1920 }
1921 if (failedRows != null) {
1922 failedRows.addAll(globalErrors.actions);
1923 }
1924 RetriesExhaustedWithDetailsException result = globalErrors.makeException(logBatchErrorDetails);
1925 globalErrors.clear();
1926 return result;
1927 }
1928
1929
1930
1931
1932 protected void incTaskCounters(Collection<byte[]> regions, ServerName sn) {
1933 tasksInProgress.incrementAndGet();
1934
1935 AtomicInteger serverCnt = taskCounterPerServer.get(sn);
1936 if (serverCnt == null) {
1937 taskCounterPerServer.putIfAbsent(sn, new AtomicInteger());
1938 serverCnt = taskCounterPerServer.get(sn);
1939 }
1940 serverCnt.incrementAndGet();
1941
1942 for (byte[] regBytes : regions) {
1943 AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
1944 if (regionCnt == null) {
1945 regionCnt = new AtomicInteger();
1946 AtomicInteger oldCnt = taskCounterPerRegion.putIfAbsent(regBytes, regionCnt);
1947 if (oldCnt != null) {
1948 regionCnt = oldCnt;
1949 }
1950 }
1951 regionCnt.incrementAndGet();
1952 }
1953 }
1954
1955
1956
1957
1958 protected void decTaskCounters(Collection<byte[]> regions, ServerName sn) {
1959 for (byte[] regBytes : regions) {
1960 AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
1961 regionCnt.decrementAndGet();
1962 }
1963
1964 taskCounterPerServer.get(sn).decrementAndGet();
1965 tasksInProgress.decrementAndGet();
1966 synchronized (tasksInProgress) {
1967 tasksInProgress.notifyAll();
1968 }
1969 }
1970
1971
1972
1973
1974
1975
1976
1977
1978 protected ConnectionManager.ServerErrorTracker createServerErrorTracker() {
1979 return new ConnectionManager.ServerErrorTracker(
1980 this.serverTrackerTimeout, this.numTries);
1981 }
1982
1983 private static boolean isReplicaGet(Row row) {
1984 return (row instanceof Get) && (((Get)row).getConsistency() == Consistency.TIMELINE);
1985 }
1986
1987
1988
1989
1990 private enum Retry {
1991 YES,
1992 NO_LOCATION_PROBLEM,
1993 NO_NOT_RETRIABLE,
1994 NO_RETRIES_EXHAUSTED,
1995 NO_OTHER_SUCCEEDED
1996 }
1997
1998
1999
2000
2001 static class RowCheckerHost {
2002 private final List<RowChecker> checkers;
2003 private boolean isEnd = false;
2004 RowCheckerHost(final List<RowChecker> checkers) {
2005 this.checkers = checkers;
2006 }
2007 void reset() throws InterruptedIOException {
2008 isEnd = false;
2009 InterruptedIOException e = null;
2010 for (RowChecker checker : checkers) {
2011 try {
2012 checker.reset();
2013 } catch (InterruptedIOException ex) {
2014 e = ex;
2015 }
2016 }
2017 if (e != null) {
2018 throw e;
2019 }
2020 }
2021 ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) {
2022 if (isEnd) {
2023 return ReturnCode.END;
2024 }
2025 ReturnCode code = ReturnCode.INCLUDE;
2026 for (RowChecker checker : checkers) {
2027 switch (checker.canTakeOperation(loc, rowSize)) {
2028 case END:
2029 isEnd = true;
2030 code = ReturnCode.END;
2031 break;
2032 case SKIP:
2033 code = ReturnCode.SKIP;
2034 break;
2035 case INCLUDE:
2036 default:
2037 break;
2038 }
2039 if (code == ReturnCode.END) {
2040 break;
2041 }
2042 }
2043 for (RowChecker checker : checkers) {
2044 checker.notifyFinal(code, loc, rowSize);
2045 }
2046 return code;
2047 }
2048 }
2049
2050
2051
2052
2053 interface RowChecker {
2054 enum ReturnCode {
2055
2056
2057
2058 INCLUDE,
2059
2060
2061
2062 SKIP,
2063
2064
2065
2066 END
2067 };
2068 ReturnCode canTakeOperation(HRegionLocation loc, long rowSize);
2069
2070
2071
2072
2073
2074 void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize);
2075
2076
2077
2078 void reset() throws InterruptedIOException ;
2079 }
2080
2081
2082
2083
2084
2085
2086 static class SubmittedSizeChecker implements RowChecker {
2087 private final long maxHeapSizeSubmit;
2088 private long heapSize = 0;
2089 SubmittedSizeChecker(final long maxHeapSizeSubmit) {
2090 this.maxHeapSizeSubmit = maxHeapSizeSubmit;
2091 }
2092 @Override
2093 public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) {
2094 if (heapSize >= maxHeapSizeSubmit) {
2095 return ReturnCode.END;
2096 }
2097 return ReturnCode.INCLUDE;
2098 }
2099
2100 @Override
2101 public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) {
2102 if (code == ReturnCode.INCLUDE) {
2103 heapSize += rowSize;
2104 }
2105 }
2106
2107 @Override
2108 public void reset() {
2109 heapSize = 0;
2110 }
2111 }
2112
2113
2114
2115 static class TaskCountChecker implements RowChecker {
2116 private static final long MAX_WAITING_TIME = 1000;
2117 private final Set<HRegionInfo> regionsIncluded = new HashSet<>();
2118 private final Set<ServerName> serversIncluded = new HashSet<>();
2119 private final int maxConcurrentTasksPerRegion;
2120 private final int maxTotalConcurrentTasks;
2121 private final int maxConcurrentTasksPerServer;
2122 private final Map<byte[], AtomicInteger> taskCounterPerRegion;
2123 private final Map<ServerName, AtomicInteger> taskCounterPerServer;
2124 private final Set<byte[]> busyRegions = new TreeSet<>(Bytes.BYTES_COMPARATOR);
2125 private final AtomicLong tasksInProgress;
2126 TaskCountChecker(final int maxTotalConcurrentTasks,
2127 final int maxConcurrentTasksPerServer,
2128 final int maxConcurrentTasksPerRegion,
2129 final AtomicLong tasksInProgress,
2130 final Map<ServerName, AtomicInteger> taskCounterPerServer,
2131 final Map<byte[], AtomicInteger> taskCounterPerRegion) {
2132 this.maxTotalConcurrentTasks = maxTotalConcurrentTasks;
2133 this.maxConcurrentTasksPerRegion = maxConcurrentTasksPerRegion;
2134 this.maxConcurrentTasksPerServer = maxConcurrentTasksPerServer;
2135 this.taskCounterPerRegion = taskCounterPerRegion;
2136 this.taskCounterPerServer = taskCounterPerServer;
2137 this.tasksInProgress = tasksInProgress;
2138 }
2139 @Override
2140 public void reset() throws InterruptedIOException {
2141
2142 waitForRegion();
2143 regionsIncluded.clear();
2144 serversIncluded.clear();
2145 busyRegions.clear();
2146 }
2147 private void waitForRegion() throws InterruptedIOException {
2148 if (busyRegions.isEmpty()) {
2149 return;
2150 }
2151 EnvironmentEdge ee = EnvironmentEdgeManager.getDelegate();
2152 final long start = ee.currentTime();
2153 while ((ee.currentTime() - start) <= MAX_WAITING_TIME) {
2154 for (byte[] region : busyRegions) {
2155 AtomicInteger count = taskCounterPerRegion.get(region);
2156 if (count == null || count.get() < maxConcurrentTasksPerRegion) {
2157 return;
2158 }
2159 }
2160 try {
2161 synchronized (tasksInProgress) {
2162 tasksInProgress.wait(10);
2163 }
2164 } catch (InterruptedException e) {
2165 throw new InterruptedIOException("Interrupted." +
2166 " tasksInProgress=" + tasksInProgress);
2167 }
2168 }
2169 }
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179 @Override
2180 public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) {
2181
2182 HRegionInfo regionInfo = loc.getRegionInfo();
2183 if (regionsIncluded.contains(regionInfo)) {
2184
2185 return ReturnCode.INCLUDE;
2186 }
2187 AtomicInteger regionCnt = taskCounterPerRegion.get(loc.getRegionInfo().getRegionName());
2188 if (regionCnt != null && regionCnt.get() >= maxConcurrentTasksPerRegion) {
2189
2190 return ReturnCode.SKIP;
2191 }
2192 int newServers = serversIncluded.size()
2193 + (serversIncluded.contains(loc.getServerName()) ? 0 : 1);
2194 if ((newServers + tasksInProgress.get()) > maxTotalConcurrentTasks) {
2195
2196 return ReturnCode.SKIP;
2197 }
2198 AtomicInteger serverCnt = taskCounterPerServer.get(loc.getServerName());
2199 if (serverCnt != null && serverCnt.get() >= maxConcurrentTasksPerServer) {
2200
2201 return ReturnCode.SKIP;
2202 }
2203 return ReturnCode.INCLUDE;
2204 }
2205
2206 @Override
2207 public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) {
2208 if (code == ReturnCode.INCLUDE) {
2209 regionsIncluded.add(loc.getRegionInfo());
2210 serversIncluded.add(loc.getServerName());
2211 }
2212 busyRegions.add(loc.getRegionInfo().getRegionName());
2213 }
2214 }
2215
2216
2217
2218
2219 static class RequestSizeChecker implements RowChecker {
2220 private final long maxHeapSizePerRequest;
2221 private final Map<ServerName, Long> serverRequestSizes = new HashMap<>();
2222 RequestSizeChecker(final long maxHeapSizePerRequest) {
2223 this.maxHeapSizePerRequest = maxHeapSizePerRequest;
2224 }
2225 @Override
2226 public void reset() {
2227 serverRequestSizes.clear();
2228 }
2229 @Override
2230 public ReturnCode canTakeOperation(HRegionLocation loc, long rowSize) {
2231
2232 long currentRequestSize = serverRequestSizes.containsKey(loc.getServerName()) ?
2233 serverRequestSizes.get(loc.getServerName()) : 0L;
2234
2235 if (currentRequestSize == 0 || currentRequestSize + rowSize <= maxHeapSizePerRequest) {
2236 return ReturnCode.INCLUDE;
2237 }
2238 return ReturnCode.SKIP;
2239 }
2240
2241 @Override
2242 public void notifyFinal(ReturnCode code, HRegionLocation loc, long rowSize) {
2243 if (code == ReturnCode.INCLUDE) {
2244 long currentRequestSize = serverRequestSizes.containsKey(loc.getServerName()) ?
2245 serverRequestSizes.get(loc.getServerName()) : 0L;
2246 serverRequestSizes.put(loc.getServerName(), currentRequestSize + rowSize);
2247 }
2248 }
2249 }
2250
2251 public static class ListRowAccess<T> implements RowAccess<T> {
2252 private final List<T> data;
2253 ListRowAccess(final List<T> data) {
2254 this.data = data;
2255 }
2256
2257 @Override
2258 public int size() {
2259 return data.size();
2260 }
2261
2262 @Override
2263 public boolean isEmpty() {
2264 return data.isEmpty();
2265 }
2266
2267 @Override
2268 public Iterator<T> iterator() {
2269 return data.iterator();
2270 }
2271 }
2272 }