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.HashMap;
28 import java.util.Iterator;
29 import java.util.List;
30 import java.util.Map;
31 import java.util.concurrent.ConcurrentHashMap;
32 import java.util.concurrent.ConcurrentMap;
33 import java.util.concurrent.ConcurrentSkipListMap;
34 import java.util.concurrent.ExecutorService;
35 import java.util.concurrent.RejectedExecutionException;
36 import java.util.concurrent.atomic.AtomicBoolean;
37 import java.util.concurrent.atomic.AtomicInteger;
38 import java.util.concurrent.atomic.AtomicLong;
39
40 import org.apache.commons.logging.Log;
41 import org.apache.commons.logging.LogFactory;
42 import org.apache.hadoop.conf.Configuration;
43 import org.apache.hadoop.hbase.DoNotRetryIOException;
44 import org.apache.hadoop.hbase.HConstants;
45 import org.apache.hadoop.hbase.HRegionInfo;
46 import org.apache.hadoop.hbase.HRegionLocation;
47 import org.apache.hadoop.hbase.ServerName;
48 import org.apache.hadoop.hbase.TableName;
49 import org.apache.hadoop.hbase.util.Bytes;
50 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
51 import org.apache.hadoop.hbase.util.Pair;
52 import org.cloudera.htrace.Trace;
53
54
55
56
57
58
59
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 class AsyncProcess<CResult> {
91 private static final Log LOG = LogFactory.getLog(AsyncProcess.class);
92 protected static final AtomicLong COUNTER = new AtomicLong();
93 protected final long id;
94 private final int startLogErrorsCnt;
95 protected final HConnection hConnection;
96 protected final TableName tableName;
97 protected final ExecutorService pool;
98 protected final AsyncProcessCallback<CResult> callback;
99 protected final BatchErrors errors = new BatchErrors();
100 protected final AtomicBoolean hasError = new AtomicBoolean(false);
101 protected final AtomicLong tasksSent = new AtomicLong(0);
102 protected final AtomicLong tasksDone = new AtomicLong(0);
103 protected final AtomicLong retriesCnt = new AtomicLong(0);
104 protected final ConcurrentMap<byte[], AtomicInteger> taskCounterPerRegion =
105 new ConcurrentSkipListMap<byte[], AtomicInteger>(Bytes.BYTES_COMPARATOR);
106 protected final ConcurrentMap<ServerName, AtomicInteger> taskCounterPerServer =
107 new ConcurrentHashMap<ServerName, AtomicInteger>();
108
109
110
111
112 protected final int maxTotalConcurrentTasks;
113
114
115
116
117
118
119
120 protected final int maxConcurrentTasksPerRegion;
121
122
123
124
125 protected final int maxConcurrentTasksPerServer;
126 protected final long pause;
127 protected int numTries;
128 protected int serverTrackerTimeout;
129 protected RpcRetryingCallerFactory rpcCallerFactory;
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145 interface AsyncProcessCallback<CResult> {
146
147
148
149
150 void success(int originalIndex, byte[] region, Row row, CResult result);
151
152
153
154
155
156
157
158
159
160 boolean failure(int originalIndex, byte[] region, Row row, Throwable t);
161
162
163
164
165
166
167
168 boolean retriableFailure(int originalIndex, Row row, byte[] region, Throwable exception);
169 }
170
171 private static class BatchErrors {
172 private final List<Throwable> throwables = new ArrayList<Throwable>();
173 private final List<Row> actions = new ArrayList<Row>();
174 private final List<String> addresses = new ArrayList<String>();
175
176 public synchronized void add(Throwable ex, Row row, HRegionLocation location) {
177 if (row == null){
178 throw new IllegalArgumentException("row cannot be null. location=" + location);
179 }
180
181 throwables.add(ex);
182 actions.add(row);
183 addresses.add(location != null ? location.getServerName().toString() : "null location");
184 }
185
186 private synchronized RetriesExhaustedWithDetailsException makeException() {
187 return new RetriesExhaustedWithDetailsException(
188 new ArrayList<Throwable>(throwables),
189 new ArrayList<Row>(actions), new ArrayList<String>(addresses));
190 }
191
192 public synchronized void clear() {
193 throwables.clear();
194 actions.clear();
195 addresses.clear();
196 }
197 }
198
199 public AsyncProcess(HConnection hc, TableName tableName, ExecutorService pool,
200 AsyncProcessCallback<CResult> callback, Configuration conf,
201 RpcRetryingCallerFactory rpcCaller) {
202 if (hc == null){
203 throw new IllegalArgumentException("HConnection cannot be null.");
204 }
205
206 this.hConnection = hc;
207 this.tableName = tableName;
208 this.pool = pool;
209 this.callback = callback;
210
211 this.id = COUNTER.incrementAndGet();
212
213 this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
214 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
215 this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
216 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
217
218 this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
219 HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS);
220 this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
221 HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS);
222 this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
223 HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS);
224
225
226
227
228 this.startLogErrorsCnt = conf.getInt("hbase.client.start.log.errors.counter", 9);
229
230 if (this.maxTotalConcurrentTasks <= 0) {
231 throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
232 }
233 if (this.maxConcurrentTasksPerServer <= 0) {
234 throw new IllegalArgumentException("maxConcurrentTasksPerServer=" +
235 maxConcurrentTasksPerServer);
236 }
237 if (this.maxConcurrentTasksPerRegion <= 0) {
238 throw new IllegalArgumentException("maxConcurrentTasksPerRegion=" +
239 maxConcurrentTasksPerRegion);
240 }
241
242
243
244
245
246
247
248
249 this.serverTrackerTimeout = 0;
250 for (int i = 0; i < this.numTries; ++i) {
251 serverTrackerTimeout += ConnectionUtils.getPauseTime(this.pause, i);
252 }
253
254
255 this.rpcCallerFactory = rpcCaller;
256 }
257
258
259
260
261
262
263
264
265 public void submit(List<? extends Row> rows, boolean atLeastOne) throws InterruptedIOException {
266 if (rows.isEmpty()) {
267 return;
268 }
269
270
271
272 Map<HRegionLocation, MultiAction<Row>> actionsByServer =
273 new HashMap<HRegionLocation, MultiAction<Row>>();
274 List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(rows.size());
275
276 long currentTaskCnt = tasksDone.get();
277 boolean alreadyLooped = false;
278
279 do {
280 if (alreadyLooped){
281
282 waitForNextTaskDone(currentTaskCnt);
283 currentTaskCnt = tasksDone.get();
284 } else {
285 alreadyLooped = true;
286 }
287
288
289 waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1);
290
291
292
293 Map<Long, Boolean> regionIncluded = new HashMap<Long, Boolean>();
294 Map<ServerName, Boolean> serverIncluded = new HashMap<ServerName, Boolean>();
295
296 int posInList = -1;
297 Iterator<? extends Row> it = rows.iterator();
298 while (it.hasNext()) {
299 Row r = it.next();
300 HRegionLocation loc = findDestLocation(r, posInList);
301
302 if (loc == null) {
303 it.remove();
304 } else if (canTakeOperation(loc, regionIncluded, serverIncluded)) {
305 Action<Row> action = new Action<Row>(r, ++posInList);
306 retainedActions.add(action);
307 addAction(loc, action, actionsByServer);
308 it.remove();
309 }
310 }
311
312 } while (retainedActions.isEmpty() && atLeastOne && !hasError());
313
314 HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
315 sendMultiAction(retainedActions, actionsByServer, 1, errorsByServer);
316 }
317
318
319
320
321
322
323
324
325 private void addAction(HRegionLocation loc, Action<Row> action, Map<HRegionLocation,
326 MultiAction<Row>> actionsByServer) {
327 final byte[] regionName = loc.getRegionInfo().getRegionName();
328 MultiAction<Row> multiAction = actionsByServer.get(loc);
329 if (multiAction == null) {
330 multiAction = new MultiAction<Row>();
331 actionsByServer.put(loc, multiAction);
332 }
333
334 multiAction.add(regionName, action);
335 }
336
337
338
339
340
341
342
343
344 private HRegionLocation findDestLocation(Row row, int posInList) {
345 if (row == null) throw new IllegalArgumentException("#" + id + ", row cannot be null");
346 HRegionLocation loc = null;
347 IOException locationException = null;
348 try {
349 loc = hConnection.locateRegion(this.tableName, row.getRow());
350 if (loc == null) {
351 locationException = new IOException("#" + id + ", no location found, aborting submit for" +
352 " tableName=" + tableName +
353 " rowkey=" + Arrays.toString(row.getRow()));
354 }
355 } catch (IOException e) {
356 locationException = e;
357 }
358 if (locationException != null) {
359
360
361 manageError(posInList, row, false, locationException, null);
362 return null;
363 }
364
365 return loc;
366 }
367
368
369
370
371
372
373
374
375
376 protected boolean canTakeOperation(HRegionLocation loc,
377 Map<Long, Boolean> regionsIncluded,
378 Map<ServerName, Boolean> serversIncluded) {
379 long regionId = loc.getRegionInfo().getRegionId();
380 Boolean regionPrevious = regionsIncluded.get(regionId);
381
382 if (regionPrevious != null) {
383
384 return regionPrevious;
385 }
386
387 Boolean serverPrevious = serversIncluded.get(loc.getServerName());
388 if (Boolean.FALSE.equals(serverPrevious)) {
389
390 regionsIncluded.put(regionId, Boolean.FALSE);
391 return false;
392 }
393
394 AtomicInteger regionCnt = taskCounterPerRegion.get(loc.getRegionInfo().getRegionName());
395 if (regionCnt != null && regionCnt.get() >= maxConcurrentTasksPerRegion) {
396
397 regionsIncluded.put(regionId, Boolean.FALSE);
398 return false;
399 }
400
401 if (serverPrevious == null) {
402
403 int newServers = 0;
404 for (Map.Entry<ServerName, Boolean> kv : serversIncluded.entrySet()) {
405 if (kv.getValue()) {
406 newServers++;
407 }
408 }
409
410
411 boolean ok = (newServers + getCurrentTasksCount()) < maxTotalConcurrentTasks;
412
413 if (ok) {
414
415 AtomicInteger serverCnt = taskCounterPerServer.get(loc.getServerName());
416 ok = (serverCnt == null || serverCnt.get() < maxConcurrentTasksPerServer);
417 }
418
419 if (!ok) {
420 regionsIncluded.put(regionId, Boolean.FALSE);
421 serversIncluded.put(loc.getServerName(), Boolean.FALSE);
422 return false;
423 }
424
425 serversIncluded.put(loc.getServerName(), Boolean.TRUE);
426 } else {
427 assert serverPrevious.equals(Boolean.TRUE);
428 }
429
430 regionsIncluded.put(regionId, Boolean.TRUE);
431
432 return true;
433 }
434
435
436
437
438
439
440
441 public void submitAll(List<? extends Row> rows) {
442 List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
443
444
445 int posInList = -1;
446 for (Row r : rows) {
447 posInList++;
448 Action<Row> action = new Action<Row>(r, posInList);
449 actions.add(action);
450 }
451 HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
452 submit(actions, actions, 1, errorsByServer);
453 }
454
455
456
457
458
459
460
461
462
463
464 private void submit(List<Action<Row>> initialActions,
465 List<Action<Row>> currentActions, int numAttempt,
466 final HConnectionManager.ServerErrorTracker errorsByServer) {
467
468 if (numAttempt > 1){
469 retriesCnt.incrementAndGet();
470 }
471
472
473 final Map<HRegionLocation, MultiAction<Row>> actionsByServer =
474 new HashMap<HRegionLocation, MultiAction<Row>>();
475
476 for (Action<Row> action : currentActions) {
477 HRegionLocation loc = findDestLocation(action.getAction(), action.getOriginalIndex());
478 if (loc != null) {
479 addAction(loc, action, actionsByServer);
480 }
481 }
482
483 if (!actionsByServer.isEmpty()) {
484 sendMultiAction(initialActions, actionsByServer, numAttempt, errorsByServer);
485 }
486 }
487
488
489
490
491
492
493
494
495
496 public void sendMultiAction(final List<Action<Row>> initialActions,
497 Map<HRegionLocation, MultiAction<Row>> actionsByServer,
498 final int numAttempt,
499 final HConnectionManager.ServerErrorTracker errorsByServer) {
500
501
502 for (Map.Entry<HRegionLocation, MultiAction<Row>> e : actionsByServer.entrySet()) {
503 final HRegionLocation loc = e.getKey();
504 final MultiAction<Row> multiAction = e.getValue();
505 incTaskCounters(multiAction.getRegions(), loc.getServerName());
506 Runnable runnable = Trace.wrap("AsyncProcess.sendMultiAction", new Runnable() {
507 @Override
508 public void run() {
509 MultiResponse res;
510 try {
511 MultiServerCallable<Row> callable = createCallable(loc, multiAction);
512 try {
513 res = createCaller(callable).callWithoutRetries(callable);
514 } catch (IOException e) {
515
516
517 receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, e,
518 errorsByServer);
519 return;
520 } catch (Throwable t) {
521
522 LOG.error("#" + id + ", Caught throwable while calling. This is unexpected." +
523 " Retrying. Server is " + loc.getServerName() + ", tableName=" + tableName, t);
524 receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, t,
525 errorsByServer);
526 return;
527 }
528
529
530 receiveMultiAction(initialActions, multiAction, loc, res, numAttempt, errorsByServer);
531
532 } finally {
533 decTaskCounters(multiAction.getRegions(), loc.getServerName());
534 }
535 }
536 });
537
538 try {
539 this.pool.submit(runnable);
540 } catch (RejectedExecutionException ree) {
541
542
543 decTaskCounters(multiAction.getRegions(), loc.getServerName());
544 LOG.warn("#" + id + ", the task was rejected by the pool. This is unexpected." +
545 " Server is " + loc.getServerName(), ree);
546
547
548 receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, ree, errorsByServer);
549 }
550 }
551 }
552
553
554
555
556 protected MultiServerCallable<Row> createCallable(final HRegionLocation location,
557 final MultiAction<Row> multi) {
558 return new MultiServerCallable<Row>(hConnection, tableName, location, multi);
559 }
560
561
562
563
564
565
566 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
567 return rpcCallerFactory.<MultiResponse> newCaller();
568 }
569
570
571
572
573
574
575
576
577
578
579
580 private boolean manageError(int originalIndex, Row row, boolean canRetry,
581 Throwable throwable, HRegionLocation location) {
582 if (canRetry && throwable != null && throwable instanceof DoNotRetryIOException) {
583 canRetry = false;
584 }
585
586 byte[] region = null;
587 if (canRetry && callback != null) {
588 region = location == null ? null : location.getRegionInfo().getEncodedNameAsBytes();
589 canRetry = callback.retriableFailure(originalIndex, row, region, throwable);
590 }
591
592 if (!canRetry) {
593 if (callback != null) {
594 if (region == null && location != null) {
595 region = location.getRegionInfo().getEncodedNameAsBytes();
596 }
597 callback.failure(originalIndex, region, row, throwable);
598 }
599 errors.add(throwable, row, location);
600 this.hasError.set(true);
601 }
602
603 return canRetry;
604 }
605
606
607
608
609
610
611
612
613
614
615 private void receiveGlobalFailure(List<Action<Row>> initialActions, MultiAction<Row> rsActions,
616 HRegionLocation location, int numAttempt, Throwable t,
617 HConnectionManager.ServerErrorTracker errorsByServer) {
618
619
620 hConnection.updateCachedLocations(tableName,
621 rsActions.actions.values().iterator().next().get(0).getAction().getRow(), null, location);
622 errorsByServer.reportServerError(location);
623
624 List<Action<Row>> toReplay = new ArrayList<Action<Row>>(initialActions.size());
625 for (Map.Entry<byte[], List<Action<Row>>> e : rsActions.actions.entrySet()) {
626 for (Action<Row> action : e.getValue()) {
627 if (manageError(action.getOriginalIndex(), action.getAction(), true, t, location)) {
628 toReplay.add(action);
629 }
630 }
631 }
632
633 logAndResubmit(initialActions, location, toReplay, numAttempt, rsActions.size(),
634 t, errorsByServer);
635 }
636
637
638
639
640
641 private void logAndResubmit(List<Action<Row>> initialActions, HRegionLocation oldLocation,
642 List<Action<Row>> toReplay, int numAttempt, int failureCount,
643 Throwable throwable,
644 HConnectionManager.ServerErrorTracker errorsByServer) {
645 if (toReplay.isEmpty()) {
646
647 if (failureCount != 0) {
648
649 LOG.warn(createLog(numAttempt, failureCount, toReplay.size(),
650 oldLocation.getServerName(), throwable, -1, false,
651 errorsByServer.getStartTrackingTime()));
652 } else if (numAttempt > startLogErrorsCnt + 1) {
653
654 LOG.info(createLog(numAttempt, failureCount, 0,
655 oldLocation.getServerName(), throwable, -1, false,
656 errorsByServer.getStartTrackingTime()));
657 }
658 return;
659 }
660
661
662
663
664
665
666
667
668 long backOffTime = errorsByServer.calculateBackoffTime(oldLocation, pause);
669
670 if (numAttempt > startLogErrorsCnt) {
671
672
673 LOG.info(createLog(numAttempt, failureCount, toReplay.size(),
674 oldLocation.getServerName(), throwable, backOffTime, true,
675 errorsByServer.getStartTrackingTime()));
676 }
677
678 try {
679 Thread.sleep(backOffTime);
680 } catch (InterruptedException e) {
681 LOG.warn("#" + id + ", not sent: " + toReplay.size() + " operations, " + oldLocation, e);
682 Thread.interrupted();
683 return;
684 }
685
686 submit(initialActions, toReplay, numAttempt + 1, errorsByServer);
687 }
688
689
690
691
692
693
694
695
696
697
698 private void receiveMultiAction(List<Action<Row>> initialActions, MultiAction<Row> multiAction,
699 HRegionLocation location,
700 MultiResponse responses, int numAttempt,
701 HConnectionManager.ServerErrorTracker errorsByServer) {
702 assert responses != null;
703
704
705
706
707
708
709
710 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
711 Throwable throwable = null;
712 int failureCount = 0;
713 boolean canRetry = true;
714
715 for (Map.Entry<byte[], List<Pair<Integer, Object>>> resultsForRS :
716 responses.getResults().entrySet()) {
717
718 boolean regionFailureRegistered = false;
719 for (Pair<Integer, Object> regionResult : resultsForRS.getValue()) {
720 Object result = regionResult.getSecond();
721
722
723 if (result == null || result instanceof Throwable) {
724 throwable = (Throwable) result;
725 Action<Row> correspondingAction = initialActions.get(regionResult.getFirst());
726 Row row = correspondingAction.getAction();
727 failureCount++;
728 if (!regionFailureRegistered) {
729 regionFailureRegistered= true;
730
731 hConnection.updateCachedLocations(this.tableName, row.getRow(), result, location);
732 if (failureCount == 1) {
733 errorsByServer.reportServerError(location);
734 canRetry = errorsByServer.canRetryMore(numAttempt);
735 }
736 }
737
738 if (manageError(correspondingAction.getOriginalIndex(), row, canRetry,
739 throwable, location)) {
740 toReplay.add(correspondingAction);
741 }
742 } else {
743 if (callback != null) {
744 int index = regionResult.getFirst();
745 Action<Row> correspondingAction = initialActions.get(index);
746 Row row = correspondingAction.getAction();
747
748 this.callback.success(index, resultsForRS.getKey(), row, (CResult) result);
749 }
750 }
751 }
752 }
753
754
755
756
757 for (Map.Entry<byte[], Throwable> throwableEntry : responses.getExceptions().entrySet()) {
758 throwable = throwableEntry.getValue();
759 byte[] region =throwableEntry.getKey();
760 List<Action<Row>> actions = multiAction.actions.get(region);
761 if (actions == null || actions.isEmpty()) {
762 throw new IllegalStateException("Wrong response for the region: " +
763 HRegionInfo.encodeRegionName(region));
764 }
765
766 if (failureCount == 0) {
767 errorsByServer.reportServerError(location);
768 canRetry = errorsByServer.canRetryMore(numAttempt);
769 }
770 hConnection.updateCachedLocations(this.tableName, actions.get(0).getAction().getRow(),
771 throwable, location);
772 failureCount += actions.size();
773
774 for (Action<Row> action : actions) {
775 Row row = action.getAction();
776 if (manageError(action.getOriginalIndex(), row, canRetry, throwable, location)) {
777 toReplay.add(action);
778 }
779 }
780 }
781
782 logAndResubmit(initialActions, location, toReplay, numAttempt, failureCount,
783 throwable, errorsByServer);
784 }
785
786 private String createLog(int numAttempt, int failureCount, int replaySize, ServerName sn,
787 Throwable error, long backOffTime, boolean willRetry, String startTime){
788 StringBuilder sb = new StringBuilder();
789
790 sb.append("#").append(id).append(", table=").append(tableName).
791 append(", attempt=").append(numAttempt).append("/").append(numTries).append(" ");
792
793 if (failureCount > 0 || error != null){
794 sb.append("failed ").append(failureCount).append(" ops").append(", last exception: ").
795 append(error == null ? "null" : error);
796 } else {
797 sb.append("SUCCEEDED");
798 }
799
800 sb.append(" on ").append(sn);
801
802 sb.append(", tracking started ").append(startTime);
803
804 if (willRetry) {
805 sb.append(", retrying after ").append(backOffTime).append(" ms").
806 append(", replay ").append(replaySize).append(" ops.");
807 } else if (failureCount > 0) {
808 sb.append(" - FAILED, NOT RETRYING ANYMORE");
809 }
810
811 return sb.toString();
812 }
813
814
815
816
817
818 protected void waitForNextTaskDone(long currentNumberOfTask) throws InterruptedIOException {
819 while (currentNumberOfTask == tasksDone.get()) {
820 try {
821 synchronized (this.tasksDone) {
822 this.tasksDone.wait(100);
823 }
824 } catch (InterruptedException e) {
825 throw new InterruptedIOException("#" + id + ", interrupted." +
826 " currentNumberOfTask=" + currentNumberOfTask +
827 ", tableName=" + tableName + ", tasksDone=" + tasksDone.get());
828 }
829 }
830 }
831
832
833
834
835 private void waitForMaximumCurrentTasks(int max) throws InterruptedIOException {
836 long lastLog = EnvironmentEdgeManager.currentTimeMillis();
837 long currentTasksDone = this.tasksDone.get();
838
839 while ((tasksSent.get() - currentTasksDone) > max) {
840 long now = EnvironmentEdgeManager.currentTimeMillis();
841 if (now > lastLog + 10000) {
842 lastLog = now;
843 LOG.info("#" + id + ", waiting for some tasks to finish. Expected max="
844 + max + ", tasksSent=" + tasksSent.get() + ", tasksDone=" + tasksDone.get() +
845 ", currentTasksDone=" + currentTasksDone + ", retries=" + retriesCnt.get() +
846 " hasError=" + hasError() + ", tableName=" + tableName);
847 }
848 waitForNextTaskDone(currentTasksDone);
849 currentTasksDone = this.tasksDone.get();
850 }
851 }
852
853 private long getCurrentTasksCount(){
854 return tasksSent.get() - tasksDone.get();
855 }
856
857
858
859
860 public void waitUntilDone() throws InterruptedIOException {
861 waitForMaximumCurrentTasks(0);
862 }
863
864
865 public boolean hasError() {
866 return hasError.get();
867 }
868
869 public List<? extends Row> getFailedOperations() {
870 return errors.actions;
871 }
872
873
874
875
876 public void clearErrors() {
877 errors.clear();
878 hasError.set(false);
879 }
880
881 public RetriesExhaustedWithDetailsException getErrors() {
882 return errors.makeException();
883 }
884
885
886
887
888 protected void incTaskCounters(Collection<byte[]> regions, ServerName sn) {
889 tasksSent.incrementAndGet();
890
891 AtomicInteger serverCnt = taskCounterPerServer.get(sn);
892 if (serverCnt == null) {
893 taskCounterPerServer.putIfAbsent(sn, new AtomicInteger());
894 serverCnt = taskCounterPerServer.get(sn);
895 }
896 serverCnt.incrementAndGet();
897
898 for (byte[] regBytes : regions) {
899 AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
900 if (regionCnt == null) {
901 regionCnt = new AtomicInteger();
902 AtomicInteger oldCnt = taskCounterPerRegion.putIfAbsent(regBytes, regionCnt);
903 if (oldCnt != null) {
904 regionCnt = oldCnt;
905 }
906 }
907 regionCnt.incrementAndGet();
908 }
909 }
910
911
912
913
914 protected void decTaskCounters(Collection<byte[]> regions, ServerName sn) {
915 for (byte[] regBytes : regions) {
916 AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
917 regionCnt.decrementAndGet();
918 }
919
920 taskCounterPerServer.get(sn).decrementAndGet();
921
922 tasksDone.incrementAndGet();
923 synchronized (tasksDone) {
924 tasksDone.notifyAll();
925 }
926 }
927
928
929
930
931
932
933
934
935 protected HConnectionManager.ServerErrorTracker createServerErrorTracker() {
936 return new HConnectionManager.ServerErrorTracker(this.serverTrackerTimeout, this.numTries);
937 }
938 }