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 NonceGenerator ng = this.hConnection.getNonceGenerator();
280 do {
281 if (alreadyLooped){
282
283 waitForNextTaskDone(currentTaskCnt);
284 currentTaskCnt = tasksDone.get();
285 } else {
286 alreadyLooped = true;
287 }
288
289
290 waitForMaximumCurrentTasks(maxTotalConcurrentTasks - 1);
291
292
293
294 Map<Long, Boolean> regionIncluded = new HashMap<Long, Boolean>();
295 Map<ServerName, Boolean> serverIncluded = new HashMap<ServerName, Boolean>();
296
297 int posInList = -1;
298 Iterator<? extends Row> it = rows.iterator();
299 while (it.hasNext()) {
300 Row r = it.next();
301 HRegionLocation loc = findDestLocation(r, posInList);
302
303 if (loc == null) {
304 it.remove();
305 } else if (canTakeOperation(loc, regionIncluded, serverIncluded)) {
306 Action<Row> action = new Action<Row>(r, ++posInList);
307 setNonce(ng, r, action);
308 retainedActions.add(action);
309 addAction(loc, action, actionsByServer, ng);
310 it.remove();
311 }
312 }
313 } while (retainedActions.isEmpty() && atLeastOne && !hasError());
314
315 HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
316 sendMultiAction(retainedActions, actionsByServer, 1, errorsByServer);
317 }
318
319
320
321
322
323
324
325
326
327 private void addAction(HRegionLocation loc, Action<Row> action, Map<HRegionLocation,
328 MultiAction<Row>> actionsByServer, NonceGenerator ng) {
329 final byte[] regionName = loc.getRegionInfo().getRegionName();
330 MultiAction<Row> multiAction = actionsByServer.get(loc);
331 if (multiAction == null) {
332 multiAction = new MultiAction<Row>();
333 actionsByServer.put(loc, multiAction);
334 }
335 if (action.hasNonce() && !multiAction.hasNonceGroup()) {
336
337
338 multiAction.setNonceGroup(ng.getNonceGroup());
339 }
340
341 multiAction.add(regionName, action);
342 }
343
344
345
346
347
348
349
350
351 private HRegionLocation findDestLocation(Row row, int posInList) {
352 if (row == null) throw new IllegalArgumentException("#" + id + ", row cannot be null");
353 HRegionLocation loc = null;
354 IOException locationException = null;
355 try {
356 loc = hConnection.locateRegion(this.tableName, row.getRow());
357 if (loc == null) {
358 locationException = new IOException("#" + id + ", no location found, aborting submit for" +
359 " tableName=" + tableName +
360 " rowkey=" + Arrays.toString(row.getRow()));
361 }
362 } catch (IOException e) {
363 locationException = e;
364 }
365 if (locationException != null) {
366
367
368 manageError(posInList, row, false, locationException, null);
369 return null;
370 }
371
372 return loc;
373 }
374
375
376
377
378
379
380
381
382
383 protected boolean canTakeOperation(HRegionLocation loc,
384 Map<Long, Boolean> regionsIncluded,
385 Map<ServerName, Boolean> serversIncluded) {
386 long regionId = loc.getRegionInfo().getRegionId();
387 Boolean regionPrevious = regionsIncluded.get(regionId);
388
389 if (regionPrevious != null) {
390
391 return regionPrevious;
392 }
393
394 Boolean serverPrevious = serversIncluded.get(loc.getServerName());
395 if (Boolean.FALSE.equals(serverPrevious)) {
396
397 regionsIncluded.put(regionId, Boolean.FALSE);
398 return false;
399 }
400
401 AtomicInteger regionCnt = taskCounterPerRegion.get(loc.getRegionInfo().getRegionName());
402 if (regionCnt != null && regionCnt.get() >= maxConcurrentTasksPerRegion) {
403
404 regionsIncluded.put(regionId, Boolean.FALSE);
405 return false;
406 }
407
408 if (serverPrevious == null) {
409
410 int newServers = 0;
411 for (Map.Entry<ServerName, Boolean> kv : serversIncluded.entrySet()) {
412 if (kv.getValue()) {
413 newServers++;
414 }
415 }
416
417
418 boolean ok = (newServers + getCurrentTasksCount()) < maxTotalConcurrentTasks;
419
420 if (ok) {
421
422 AtomicInteger serverCnt = taskCounterPerServer.get(loc.getServerName());
423 ok = (serverCnt == null || serverCnt.get() < maxConcurrentTasksPerServer);
424 }
425
426 if (!ok) {
427 regionsIncluded.put(regionId, Boolean.FALSE);
428 serversIncluded.put(loc.getServerName(), Boolean.FALSE);
429 return false;
430 }
431
432 serversIncluded.put(loc.getServerName(), Boolean.TRUE);
433 } else {
434 assert serverPrevious.equals(Boolean.TRUE);
435 }
436
437 regionsIncluded.put(regionId, Boolean.TRUE);
438
439 return true;
440 }
441
442
443
444
445
446
447
448 public void submitAll(List<? extends Row> rows) {
449 List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
450
451
452 int posInList = -1;
453 NonceGenerator ng = this.hConnection.getNonceGenerator();
454 for (Row r : rows) {
455 posInList++;
456 Action<Row> action = new Action<Row>(r, posInList);
457 setNonce(ng, r, action);
458 actions.add(action);
459 }
460 HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
461 submit(actions, actions, 1, errorsByServer);
462 }
463
464 private void setNonce(NonceGenerator ng, Row r, Action<Row> action) {
465 if (!(r instanceof Append) && !(r instanceof Increment)) return;
466 action.setNonce(ng.newNonce());
467 }
468
469
470
471
472
473
474
475
476
477
478 private void submit(List<Action<Row>> initialActions,
479 List<Action<Row>> currentActions, int numAttempt,
480 final HConnectionManager.ServerErrorTracker errorsByServer) {
481
482 if (numAttempt > 1){
483 retriesCnt.incrementAndGet();
484 }
485
486
487 final Map<HRegionLocation, MultiAction<Row>> actionsByServer =
488 new HashMap<HRegionLocation, MultiAction<Row>>();
489
490 NonceGenerator ng = this.hConnection.getNonceGenerator();
491 for (Action<Row> action : currentActions) {
492 HRegionLocation loc = findDestLocation(action.getAction(), action.getOriginalIndex());
493 if (loc != null) {
494 addAction(loc, action, actionsByServer, ng);
495 }
496 }
497
498 if (!actionsByServer.isEmpty()) {
499 sendMultiAction(initialActions, actionsByServer, numAttempt, errorsByServer);
500 }
501 }
502
503
504
505
506
507
508
509
510
511 public void sendMultiAction(final List<Action<Row>> initialActions,
512 Map<HRegionLocation, MultiAction<Row>> actionsByServer,
513 final int numAttempt,
514 final HConnectionManager.ServerErrorTracker errorsByServer) {
515
516
517 for (Map.Entry<HRegionLocation, MultiAction<Row>> e : actionsByServer.entrySet()) {
518 final HRegionLocation loc = e.getKey();
519 final MultiAction<Row> multiAction = e.getValue();
520 incTaskCounters(multiAction.getRegions(), loc.getServerName());
521 Runnable runnable = Trace.wrap("AsyncProcess.sendMultiAction", new Runnable() {
522 @Override
523 public void run() {
524 MultiResponse res;
525 try {
526 MultiServerCallable<Row> callable = createCallable(loc, multiAction);
527 try {
528 res = createCaller(callable).callWithoutRetries(callable);
529 } catch (IOException e) {
530
531
532 receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, e,
533 errorsByServer);
534 return;
535 } catch (Throwable t) {
536
537 LOG.error("#" + id + ", Caught throwable while calling. This is unexpected." +
538 " Retrying. Server is " + loc.getServerName() + ", tableName=" + tableName, t);
539 receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, t,
540 errorsByServer);
541 return;
542 }
543
544
545 receiveMultiAction(initialActions, multiAction, loc, res, numAttempt, errorsByServer);
546
547 } finally {
548 decTaskCounters(multiAction.getRegions(), loc.getServerName());
549 }
550 }
551 });
552
553 try {
554 this.pool.submit(runnable);
555 } catch (RejectedExecutionException ree) {
556
557
558 decTaskCounters(multiAction.getRegions(), loc.getServerName());
559 LOG.warn("#" + id + ", the task was rejected by the pool. This is unexpected." +
560 " Server is " + loc.getServerName(), ree);
561
562
563 receiveGlobalFailure(initialActions, multiAction, loc, numAttempt, ree, errorsByServer);
564 }
565 }
566 }
567
568
569
570
571 protected MultiServerCallable<Row> createCallable(final HRegionLocation location,
572 final MultiAction<Row> multi) {
573 return new MultiServerCallable<Row>(hConnection, tableName, location, multi);
574 }
575
576
577
578
579
580
581 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
582 return rpcCallerFactory.<MultiResponse> newCaller();
583 }
584
585
586
587
588
589
590
591
592
593
594
595 private boolean manageError(int originalIndex, Row row, boolean canRetry,
596 Throwable throwable, HRegionLocation location) {
597 if (canRetry && throwable != null && throwable instanceof DoNotRetryIOException) {
598 canRetry = false;
599 }
600
601 byte[] region = null;
602 if (canRetry && callback != null) {
603 region = location == null ? null : location.getRegionInfo().getEncodedNameAsBytes();
604 canRetry = callback.retriableFailure(originalIndex, row, region, throwable);
605 }
606
607 if (!canRetry) {
608 if (callback != null) {
609 if (region == null && location != null) {
610 region = location.getRegionInfo().getEncodedNameAsBytes();
611 }
612 callback.failure(originalIndex, region, row, throwable);
613 }
614 errors.add(throwable, row, location);
615 this.hasError.set(true);
616 }
617
618 return canRetry;
619 }
620
621
622
623
624
625
626
627
628
629
630 private void receiveGlobalFailure(List<Action<Row>> initialActions, MultiAction<Row> rsActions,
631 HRegionLocation location, int numAttempt, Throwable t,
632 HConnectionManager.ServerErrorTracker errorsByServer) {
633
634
635 hConnection.updateCachedLocations(tableName,
636 rsActions.actions.values().iterator().next().get(0).getAction().getRow(), null, location);
637 errorsByServer.reportServerError(location);
638
639 List<Action<Row>> toReplay = new ArrayList<Action<Row>>(initialActions.size());
640 for (Map.Entry<byte[], List<Action<Row>>> e : rsActions.actions.entrySet()) {
641 for (Action<Row> action : e.getValue()) {
642 if (manageError(action.getOriginalIndex(), action.getAction(), true, t, location)) {
643 toReplay.add(action);
644 }
645 }
646 }
647
648 logAndResubmit(initialActions, location, toReplay, numAttempt, rsActions.size(),
649 t, errorsByServer);
650 }
651
652
653
654
655
656 private void logAndResubmit(List<Action<Row>> initialActions, HRegionLocation oldLocation,
657 List<Action<Row>> toReplay, int numAttempt, int failureCount,
658 Throwable throwable,
659 HConnectionManager.ServerErrorTracker errorsByServer) {
660 if (toReplay.isEmpty()) {
661
662 if (failureCount != 0) {
663
664 LOG.warn(createLog(numAttempt, failureCount, toReplay.size(),
665 oldLocation.getServerName(), throwable, -1, false,
666 errorsByServer.getStartTrackingTime()));
667 } else if (numAttempt > startLogErrorsCnt + 1) {
668
669 LOG.info(createLog(numAttempt, failureCount, 0,
670 oldLocation.getServerName(), throwable, -1, false,
671 errorsByServer.getStartTrackingTime()));
672 }
673 return;
674 }
675
676
677
678
679
680
681
682
683 long backOffTime = errorsByServer.calculateBackoffTime(oldLocation, pause);
684
685 if (numAttempt > startLogErrorsCnt) {
686
687
688 LOG.info(createLog(numAttempt, failureCount, toReplay.size(),
689 oldLocation.getServerName(), throwable, backOffTime, true,
690 errorsByServer.getStartTrackingTime()));
691 }
692
693 try {
694 Thread.sleep(backOffTime);
695 } catch (InterruptedException e) {
696 LOG.warn("#" + id + ", not sent: " + toReplay.size() + " operations, " + oldLocation, e);
697 Thread.currentThread().interrupt();
698 return;
699 }
700
701 submit(initialActions, toReplay, numAttempt + 1, errorsByServer);
702 }
703
704
705
706
707
708
709
710
711
712
713 private void receiveMultiAction(List<Action<Row>> initialActions, MultiAction<Row> multiAction,
714 HRegionLocation location,
715 MultiResponse responses, int numAttempt,
716 HConnectionManager.ServerErrorTracker errorsByServer) {
717 assert responses != null;
718
719
720
721
722
723
724
725 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
726 Throwable throwable = null;
727 int failureCount = 0;
728 boolean canRetry = true;
729
730 for (Map.Entry<byte[], List<Pair<Integer, Object>>> resultsForRS :
731 responses.getResults().entrySet()) {
732
733 boolean regionFailureRegistered = false;
734 for (Pair<Integer, Object> regionResult : resultsForRS.getValue()) {
735 Object result = regionResult.getSecond();
736
737
738 if (result == null || result instanceof Throwable) {
739 throwable = (Throwable) result;
740 Action<Row> correspondingAction = initialActions.get(regionResult.getFirst());
741 Row row = correspondingAction.getAction();
742 failureCount++;
743 if (!regionFailureRegistered) {
744 regionFailureRegistered= true;
745
746 hConnection.updateCachedLocations(this.tableName, row.getRow(), result, location);
747 if (failureCount == 1) {
748 errorsByServer.reportServerError(location);
749 canRetry = errorsByServer.canRetryMore(numAttempt);
750 }
751 }
752
753 if (manageError(correspondingAction.getOriginalIndex(), row, canRetry,
754 throwable, location)) {
755 toReplay.add(correspondingAction);
756 }
757 } else {
758 if (callback != null) {
759 int index = regionResult.getFirst();
760 Action<Row> correspondingAction = initialActions.get(index);
761 Row row = correspondingAction.getAction();
762
763 this.callback.success(index, resultsForRS.getKey(), row, (CResult) result);
764 }
765 }
766 }
767 }
768
769
770
771
772 for (Map.Entry<byte[], Throwable> throwableEntry : responses.getExceptions().entrySet()) {
773 throwable = throwableEntry.getValue();
774 byte[] region =throwableEntry.getKey();
775 List<Action<Row>> actions = multiAction.actions.get(region);
776 if (actions == null || actions.isEmpty()) {
777 throw new IllegalStateException("Wrong response for the region: " +
778 HRegionInfo.encodeRegionName(region));
779 }
780
781 if (failureCount == 0) {
782 errorsByServer.reportServerError(location);
783 canRetry = errorsByServer.canRetryMore(numAttempt);
784 }
785 hConnection.updateCachedLocations(this.tableName, actions.get(0).getAction().getRow(),
786 throwable, location);
787 failureCount += actions.size();
788
789 for (Action<Row> action : actions) {
790 Row row = action.getAction();
791 if (manageError(action.getOriginalIndex(), row, canRetry, throwable, location)) {
792 toReplay.add(action);
793 }
794 }
795 }
796
797 logAndResubmit(initialActions, location, toReplay, numAttempt, failureCount,
798 throwable, errorsByServer);
799 }
800
801 private String createLog(int numAttempt, int failureCount, int replaySize, ServerName sn,
802 Throwable error, long backOffTime, boolean willRetry, String startTime){
803 StringBuilder sb = new StringBuilder();
804
805 sb.append("#").append(id).append(", table=").append(tableName).
806 append(", attempt=").append(numAttempt).append("/").append(numTries).append(" ");
807
808 if (failureCount > 0 || error != null){
809 sb.append("failed ").append(failureCount).append(" ops").append(", last exception: ").
810 append(error == null ? "null" : error);
811 } else {
812 sb.append("SUCCEEDED");
813 }
814
815 sb.append(" on ").append(sn);
816
817 sb.append(", tracking started ").append(startTime);
818
819 if (willRetry) {
820 sb.append(", retrying after ").append(backOffTime).append(" ms").
821 append(", replay ").append(replaySize).append(" ops.");
822 } else if (failureCount > 0) {
823 sb.append(" - FAILED, NOT RETRYING ANYMORE");
824 }
825
826 return sb.toString();
827 }
828
829
830
831
832
833 protected void waitForNextTaskDone(long currentNumberOfTask) throws InterruptedIOException {
834 while (currentNumberOfTask == tasksDone.get()) {
835 try {
836 synchronized (this.tasksDone) {
837 this.tasksDone.wait(100);
838 }
839 } catch (InterruptedException e) {
840 throw new InterruptedIOException("#" + id + ", interrupted." +
841 " currentNumberOfTask=" + currentNumberOfTask +
842 ", tableName=" + tableName + ", tasksDone=" + tasksDone.get());
843 }
844 }
845 }
846
847
848
849
850 private void waitForMaximumCurrentTasks(int max) throws InterruptedIOException {
851 long lastLog = EnvironmentEdgeManager.currentTimeMillis();
852 long currentTasksDone = this.tasksDone.get();
853
854 while ((tasksSent.get() - currentTasksDone) > max) {
855 long now = EnvironmentEdgeManager.currentTimeMillis();
856 if (now > lastLog + 10000) {
857 lastLog = now;
858 LOG.info("#" + id + ", waiting for some tasks to finish. Expected max="
859 + max + ", tasksSent=" + tasksSent.get() + ", tasksDone=" + tasksDone.get() +
860 ", currentTasksDone=" + currentTasksDone + ", retries=" + retriesCnt.get() +
861 " hasError=" + hasError() + ", tableName=" + tableName);
862 }
863 waitForNextTaskDone(currentTasksDone);
864 currentTasksDone = this.tasksDone.get();
865 }
866 }
867
868 private long getCurrentTasksCount(){
869 return tasksSent.get() - tasksDone.get();
870 }
871
872
873
874
875 public void waitUntilDone() throws InterruptedIOException {
876 waitForMaximumCurrentTasks(0);
877 }
878
879
880 public boolean hasError() {
881 return hasError.get();
882 }
883
884 public List<? extends Row> getFailedOperations() {
885 return errors.actions;
886 }
887
888
889
890
891 public void clearErrors() {
892 errors.clear();
893 hasError.set(false);
894 }
895
896 public RetriesExhaustedWithDetailsException getErrors() {
897 return errors.makeException();
898 }
899
900
901
902
903 protected void incTaskCounters(Collection<byte[]> regions, ServerName sn) {
904 tasksSent.incrementAndGet();
905
906 AtomicInteger serverCnt = taskCounterPerServer.get(sn);
907 if (serverCnt == null) {
908 taskCounterPerServer.putIfAbsent(sn, new AtomicInteger());
909 serverCnt = taskCounterPerServer.get(sn);
910 }
911 serverCnt.incrementAndGet();
912
913 for (byte[] regBytes : regions) {
914 AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
915 if (regionCnt == null) {
916 regionCnt = new AtomicInteger();
917 AtomicInteger oldCnt = taskCounterPerRegion.putIfAbsent(regBytes, regionCnt);
918 if (oldCnt != null) {
919 regionCnt = oldCnt;
920 }
921 }
922 regionCnt.incrementAndGet();
923 }
924 }
925
926
927
928
929 protected void decTaskCounters(Collection<byte[]> regions, ServerName sn) {
930 for (byte[] regBytes : regions) {
931 AtomicInteger regionCnt = taskCounterPerRegion.get(regBytes);
932 regionCnt.decrementAndGet();
933 }
934
935 taskCounterPerServer.get(sn).decrementAndGet();
936
937 tasksDone.incrementAndGet();
938 synchronized (tasksDone) {
939 tasksDone.notifyAll();
940 }
941 }
942
943
944
945
946
947
948
949
950 protected HConnectionManager.ServerErrorTracker createServerErrorTracker() {
951 return new HConnectionManager.ServerErrorTracker(this.serverTrackerTimeout, this.numTries);
952 }
953 }