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