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