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
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.TableName;
27 import org.apache.hadoop.hbase.HConstants;
28 import org.apache.hadoop.hbase.HRegionLocation;
29 import org.apache.hadoop.hbase.DoNotRetryIOException;
30 import org.apache.hadoop.hbase.HConstants;
31 import org.apache.hadoop.hbase.HRegionLocation;
32 import org.apache.hadoop.hbase.protobuf.generated.Tracing;
33 import org.apache.hadoop.hbase.util.Bytes;
34 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
35 import org.apache.hadoop.hbase.util.Pair;
36 import org.cloudera.htrace.Span;
37 import org.cloudera.htrace.Trace;
38
39 import java.io.IOException;
40 import java.io.InterruptedIOException;
41 import java.util.ArrayList;
42 import java.util.Arrays;
43 import java.util.HashMap;
44 import java.util.Iterator;
45 import java.util.List;
46 import java.util.Map;
47 import java.util.concurrent.ConcurrentHashMap;
48 import java.util.concurrent.ConcurrentMap;
49 import java.util.concurrent.ExecutorService;
50 import java.util.concurrent.RejectedExecutionException;
51 import java.util.concurrent.atomic.AtomicBoolean;
52 import java.util.concurrent.atomic.AtomicInteger;
53 import java.util.concurrent.atomic.AtomicLong;
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
91 class AsyncProcess<CResult> {
92 private static final Log LOG = LogFactory.getLog(AsyncProcess.class);
93 protected final HConnection hConnection;
94 protected final TableName tableName;
95 protected final ExecutorService pool;
96 protected final AsyncProcessCallback<CResult> callback;
97 protected final BatchErrors errors = new BatchErrors();
98 protected final BatchErrors retriedErrors = new BatchErrors();
99 protected final AtomicBoolean hasError = new AtomicBoolean(false);
100 protected final AtomicLong tasksSent = new AtomicLong(0);
101 protected final AtomicLong tasksDone = new AtomicLong(0);
102 protected final ConcurrentMap<String, AtomicInteger> taskCounterPerRegion =
103 new ConcurrentHashMap<String, AtomicInteger>();
104 protected final int maxTotalConcurrentTasks;
105 protected final int maxConcurrentTasksPerRegion;
106 protected final long pause;
107 protected int numTries;
108 protected final boolean useServerTrackerForRetries;
109 protected int serverTrackerTimeout;
110 protected RpcRetryingCallerFactory rpcCallerFactory;
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126 interface AsyncProcessCallback<CResult> {
127
128
129
130
131 void success(int originalIndex, byte[] region, Row row, CResult result);
132
133
134
135
136
137
138
139
140
141 boolean failure(int originalIndex, byte[] region, Row row, Throwable t);
142
143
144
145
146
147
148
149 boolean retriableFailure(int originalIndex, Row row, byte[] region, Throwable exception);
150 }
151
152 private static class BatchErrors {
153 private List<Throwable> throwables = new ArrayList<Throwable>();
154 private List<Row> actions = new ArrayList<Row>();
155 private List<String> addresses = new ArrayList<String>();
156
157 public void add(Throwable ex, Row row, HRegionLocation location) {
158 throwables.add(ex);
159 actions.add(row);
160 addresses.add(location != null ? location.getHostnamePort() : "null location");
161 }
162
163 private RetriesExhaustedWithDetailsException makeException() {
164 return new RetriesExhaustedWithDetailsException(
165 new ArrayList<Throwable>(throwables),
166 new ArrayList<Row>(actions), new ArrayList<String>(addresses));
167 }
168
169 public void clear() {
170 throwables.clear();
171 actions.clear();
172 addresses.clear();
173 }
174 }
175
176 public AsyncProcess(HConnection hc, TableName tableName, ExecutorService pool,
177 AsyncProcessCallback<CResult> callback, Configuration conf,
178 RpcRetryingCallerFactory rpcCaller) {
179 this.hConnection = hc;
180 this.tableName = tableName;
181 this.pool = pool;
182 this.callback = callback;
183
184 this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
185 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
186 this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
187 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
188
189 this.maxTotalConcurrentTasks = conf.getInt("hbase.client.max.total.tasks", 200);
190
191
192
193
194 this.maxConcurrentTasksPerRegion = conf.getInt("hbase.client.max.perregion.tasks", 1);
195
196 this.useServerTrackerForRetries =
197 conf.getBoolean(HConnectionManager.RETRIES_BY_SERVER_KEY, true);
198
199 if (this.useServerTrackerForRetries) {
200
201
202
203
204
205
206
207 this.serverTrackerTimeout = 0;
208 for (int i = 0; i < this.numTries; ++i) {
209 serverTrackerTimeout += ConnectionUtils.getPauseTime(this.pause, i);
210 }
211 }
212
213 this.rpcCallerFactory = rpcCaller;
214 }
215
216
217
218
219
220
221
222
223 public void submit(List<? extends Row> rows, boolean atLeastOne) throws InterruptedIOException {
224 if (rows.isEmpty()){
225 return;
226 }
227
228 Map<HRegionLocation, MultiAction<Row>> actionsByServer =
229 new HashMap<HRegionLocation, MultiAction<Row>>();
230 List<Action<Row>> retainedActions = new ArrayList<Action<Row>>(rows.size());
231
232 do {
233 Map<String, Boolean> regionIncluded = new HashMap<String, Boolean>();
234 long currentTaskNumber = waitForMaximumCurrentTasks(maxTotalConcurrentTasks);
235 int posInList = -1;
236 Iterator<? extends Row> it = rows.iterator();
237 while (it.hasNext()) {
238 Row r = it.next();
239 HRegionLocation loc = findDestLocation(r, 1, posInList, false, regionIncluded);
240
241 if (loc != null) {
242 Action<Row> action = new Action<Row>(r, ++posInList);
243 retainedActions.add(action);
244 addAction(loc, action, actionsByServer);
245 it.remove();
246 }
247 }
248
249 if (retainedActions.isEmpty() && atLeastOne && !hasError()) {
250 waitForNextTaskDone(currentTaskNumber);
251 }
252
253 } while (retainedActions.isEmpty() && atLeastOne && !hasError());
254
255 HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
256 sendMultiAction(retainedActions, actionsByServer, 1, errorsByServer);
257 }
258
259
260
261
262
263
264
265
266 private void addAction(HRegionLocation loc, Action<Row> action, Map<HRegionLocation,
267 MultiAction<Row>> actionsByServer) {
268 final byte[] regionName = loc.getRegionInfo().getRegionName();
269 MultiAction<Row> multiAction = actionsByServer.get(loc);
270 if (multiAction == null) {
271 multiAction = new MultiAction<Row>();
272 actionsByServer.put(loc, multiAction);
273 }
274
275 multiAction.add(regionName, action);
276 }
277
278
279
280
281
282
283
284
285
286
287
288 private HRegionLocation findDestLocation(Row row, int numAttempt,
289 int posInList, boolean force,
290 Map<String, Boolean> regionStatus) {
291 HRegionLocation loc = null;
292 IOException locationException = null;
293 try {
294 loc = hConnection.locateRegion(this.tableName, row.getRow());
295 if (loc == null) {
296 locationException = new IOException("No location found, aborting submit for" +
297 " tableName=" + tableName +
298 " rowkey=" + Arrays.toString(row.getRow()));
299 }
300 } catch (IOException e) {
301 locationException = e;
302 }
303 if (locationException != null) {
304
305
306 manageError(numAttempt, posInList, row, false, locationException, null);
307 return null;
308 }
309
310 if (force) {
311 return loc;
312 }
313
314 String regionName = loc.getRegionInfo().getEncodedName();
315 Boolean addIt = regionStatus.get(regionName);
316 if (addIt == null) {
317 addIt = canTakeNewOperations(regionName);
318 regionStatus.put(regionName, addIt);
319 }
320
321 return addIt ? loc : null;
322 }
323
324
325
326
327
328
329
330
331 protected boolean canTakeNewOperations(String encodedRegionName) {
332 AtomicInteger ct = taskCounterPerRegion.get(encodedRegionName);
333 return ct == null || ct.get() < maxConcurrentTasksPerRegion;
334 }
335
336
337
338
339
340
341
342 public void submitAll(List<? extends Row> rows) {
343 List<Action<Row>> actions = new ArrayList<Action<Row>>(rows.size());
344
345
346 int posInList = -1;
347 for (Row r : rows) {
348 posInList++;
349 Action<Row> action = new Action<Row>(r, posInList);
350 actions.add(action);
351 }
352 HConnectionManager.ServerErrorTracker errorsByServer = createServerErrorTracker();
353 submit(actions, actions, 1, true, errorsByServer);
354 }
355
356
357
358
359
360
361
362
363
364
365
366 private void submit(List<Action<Row>> initialActions,
367 List<Action<Row>> currentActions, int numAttempt, boolean force,
368 final HConnectionManager.ServerErrorTracker errorsByServer) {
369
370 final Map<HRegionLocation, MultiAction<Row>> actionsByServer =
371 new HashMap<HRegionLocation, MultiAction<Row>>();
372
373
374
375
376 Map<String, Boolean> regionIncluded = new HashMap<String, Boolean>();
377
378 for (Action<Row> action : currentActions) {
379 HRegionLocation loc = findDestLocation(
380 action.getAction(), 1, action.getOriginalIndex(), force, regionIncluded);
381
382 if (loc != null) {
383 addAction(loc, action, actionsByServer);
384 }
385 }
386
387 if (!actionsByServer.isEmpty()) {
388 sendMultiAction(initialActions, actionsByServer, numAttempt, errorsByServer);
389 }
390 }
391
392
393
394
395
396
397
398
399
400 public void sendMultiAction(final List<Action<Row>> initialActions,
401 Map<HRegionLocation, MultiAction<Row>> actionsByServer,
402 final int numAttempt,
403 final HConnectionManager.ServerErrorTracker errorsByServer) {
404
405
406 for (Map.Entry<HRegionLocation, MultiAction<Row>> e : actionsByServer.entrySet()) {
407 final HRegionLocation loc = e.getKey();
408 final MultiAction<Row> multi = e.getValue();
409 final String regionName = loc.getRegionInfo().getEncodedName();
410
411 incTaskCounters(regionName);
412
413 Runnable runnable = Trace.wrap("AsyncProcess.sendMultiAction", new Runnable() {
414 @Override
415 public void run() {
416 MultiResponse res;
417 try {
418 MultiServerCallable<Row> callable = createCallable(loc, multi);
419 try {
420 res = createCaller(callable).callWithoutRetries(callable);
421 } catch (IOException e) {
422 LOG.warn("The call to the RS failed, we don't know where we stand. location="
423 + loc, e);
424 resubmitAll(initialActions, multi, loc, numAttempt + 1, e, errorsByServer);
425 return;
426 }
427
428 receiveMultiAction(initialActions, multi, loc, res, numAttempt, errorsByServer);
429 } finally {
430 decTaskCounters(regionName);
431 }
432 }
433 });
434
435 try {
436 this.pool.submit(runnable);
437 } catch (RejectedExecutionException ree) {
438
439
440 decTaskCounters(regionName);
441 LOG.warn("The task was rejected by the pool. This is unexpected. " +
442 "location=" + loc, ree);
443
444
445 resubmitAll(initialActions, multi, loc, numAttempt + 1, ree, errorsByServer);
446 }
447 }
448 }
449
450
451
452
453 protected MultiServerCallable<Row> createCallable(final HRegionLocation location,
454 final MultiAction<Row> multi) {
455 return new MultiServerCallable<Row>(hConnection, tableName, location, multi);
456 }
457
458
459
460
461
462
463 protected RpcRetryingCaller<MultiResponse> createCaller(MultiServerCallable<Row> callable) {
464
465 return rpcCallerFactory.<MultiResponse> newCaller();
466 }
467
468
469
470
471
472
473
474
475
476
477
478
479 private boolean manageError(int numAttempt, int originalIndex, Row row, boolean canRetry,
480 Throwable throwable, HRegionLocation location) {
481 if (canRetry) {
482 if (numAttempt >= numTries ||
483 (throwable != null && throwable instanceof DoNotRetryIOException)) {
484 canRetry = false;
485 }
486 }
487 byte[] region = location == null ? null : location.getRegionInfo().getEncodedNameAsBytes();
488
489 if (canRetry && callback != null) {
490 canRetry = callback.retriableFailure(originalIndex, row, region, throwable);
491 }
492
493 if (canRetry) {
494 if (LOG.isTraceEnabled()) {
495 retriedErrors.add(throwable, row, location);
496 }
497 } else {
498 if (callback != null) {
499 callback.failure(originalIndex, region, row, throwable);
500 }
501 this.hasError.set(true);
502 errors.add(throwable, row, location);
503 }
504
505 return canRetry;
506 }
507
508
509
510
511
512
513
514
515
516
517 private void resubmitAll(List<Action<Row>> initialActions, MultiAction<Row> rsActions,
518 HRegionLocation location, int numAttempt, Throwable t,
519 HConnectionManager.ServerErrorTracker errorsByServer) {
520
521
522 hConnection.updateCachedLocations(tableName,
523 rsActions.actions.values().iterator().next().get(0).getAction().getRow(), null, location);
524 errorsByServer.reportServerError(location);
525
526 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
527 for (List<Action<Row>> actions : rsActions.actions.values()) {
528 for (Action<Row> action : actions) {
529 if (manageError(numAttempt, action.getOriginalIndex(), action.getAction(),
530 true, t, location)) {
531 toReplay.add(action);
532 }
533 }
534 }
535
536 if (toReplay.isEmpty()) {
537 LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for all (" +
538 initialActions.size() + ") operations on server " + location.getServerName() +
539 " NOT resubmitting, tableName=" + tableName + ", location=" + location);
540 } else {
541 submit(initialActions, toReplay, numAttempt, true, errorsByServer);
542 }
543 }
544
545
546
547
548
549
550
551
552
553
554 private void receiveMultiAction(List<Action<Row>> initialActions,
555 MultiAction<Row> rsActions, HRegionLocation location,
556 MultiResponse responses, int numAttempt,
557 HConnectionManager.ServerErrorTracker errorsByServer) {
558
559 if (responses == null) {
560 LOG.info("Attempt #" + numAttempt + "/" + numTries + " failed for all operations" +
561 " on server " + location.getServerName() + " , trying to resubmit," +
562 " tableName=" + tableName + ", location=" + location);
563 resubmitAll(initialActions, rsActions, location, numAttempt + 1, null, errorsByServer);
564 return;
565 }
566
567
568
569
570
571
572
573 List<Action<Row>> toReplay = new ArrayList<Action<Row>>();
574 Throwable throwable = null;
575
576 int failureCount = 0;
577 boolean canRetry = true;
578 for (Map.Entry<byte[], List<Pair<Integer, Object>>> resultsForRS :
579 responses.getResults().entrySet()) {
580
581 for (Pair<Integer, Object> regionResult : resultsForRS.getValue()) {
582 Object result = regionResult.getSecond();
583
584
585 if (result == null || result instanceof Throwable) {
586 throwable = (Throwable) result;
587 Action<Row> correspondingAction = initialActions.get(regionResult.getFirst());
588 Row row = correspondingAction.getAction();
589
590 if (failureCount++ == 0) {
591 hConnection.updateCachedLocations(this.tableName, row.getRow(), result, location);
592 if (errorsByServer != null) {
593 errorsByServer.reportServerError(location);
594 canRetry = errorsByServer.canRetryMore();
595 }
596 }
597
598 if (manageError(numAttempt, correspondingAction.getOriginalIndex(), row, canRetry,
599 throwable, location)) {
600 toReplay.add(correspondingAction);
601 }
602 } else {
603 if (callback != null) {
604 Action<Row> correspondingAction = initialActions.get(regionResult.getFirst());
605 Row row = correspondingAction.getAction();
606
607 this.callback.success(correspondingAction.getOriginalIndex(),
608 resultsForRS.getKey(), row, (CResult) result);
609 }
610 }
611 }
612 }
613
614 if (!toReplay.isEmpty()) {
615 long backOffTime = (errorsByServer != null ?
616 errorsByServer.calculateBackoffTime(location, pause) :
617 ConnectionUtils.getPauseTime(pause, numAttempt));
618 if (numAttempt > 3 && LOG.isDebugEnabled()) {
619
620
621 LOG.debug("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount +
622 " operations on server " + location.getServerName() + ", resubmitting " +
623 toReplay.size() + ", tableName=" + tableName + ", location=" +
624 location + ", last exception was: " + throwable +
625 " - sleeping " + backOffTime + " ms.");
626 }
627 try {
628 Thread.sleep(backOffTime);
629 } catch (InterruptedException e) {
630 LOG.warn("Not sent: " + toReplay.size() +
631 " operations, tableName=" + tableName + ", location=" + location, e);
632 Thread.interrupted();
633 return;
634 }
635
636 submit(initialActions, toReplay, numAttempt + 1, true, errorsByServer);
637 } else if (failureCount != 0) {
638 LOG.warn("Attempt #" + numAttempt + "/" + numTries + " failed for " + failureCount +
639 " operations on server " + location.getServerName() + " NOT resubmitting." +
640 ", tableName=" + tableName + ", location=" + location);
641 }
642 }
643
644
645
646
647
648 protected void waitForNextTaskDone(long currentNumberOfTask) throws InterruptedIOException {
649 while (currentNumberOfTask == tasksDone.get()) {
650 try {
651 synchronized (this.tasksDone) {
652 this.tasksDone.wait(100);
653 }
654 } catch (InterruptedException e) {
655 throw new InterruptedIOException("Interrupted." +
656 " currentNumberOfTask=" + currentNumberOfTask +
657 ", tableName=" + tableName + ", tasksDone=" + tasksDone.get());
658 }
659 }
660 }
661
662
663
664
665 private long waitForMaximumCurrentTasks(int max) throws InterruptedIOException {
666 long lastLog = EnvironmentEdgeManager.currentTimeMillis();
667 long currentTasksDone = this.tasksDone.get();
668
669 while ((tasksSent.get() - currentTasksDone) > max) {
670 long now = EnvironmentEdgeManager.currentTimeMillis();
671 if (now > lastLog + 10000) {
672 lastLog = now;
673 LOG.info(": Waiting for the global number of running tasks to be equals or less than "
674 + max + ", tasksSent=" + tasksSent.get() + ", tasksDone=" + tasksDone.get() +
675 ", currentTasksDone=" + currentTasksDone + ", tableName=" + tableName);
676 }
677 waitForNextTaskDone(currentTasksDone);
678 currentTasksDone = this.tasksDone.get();
679 }
680
681 return currentTasksDone;
682 }
683
684
685
686
687 public void waitUntilDone() throws InterruptedIOException {
688 waitForMaximumCurrentTasks(0);
689 }
690
691
692 public boolean hasError() {
693 return hasError.get();
694 }
695
696 public List<? extends Row> getFailedOperations() {
697 return errors.actions;
698 }
699
700
701
702
703 public void clearErrors() {
704 errors.clear();
705 retriedErrors.clear();
706 hasError.set(false);
707 }
708
709 public RetriesExhaustedWithDetailsException getErrors() {
710 return errors.makeException();
711 }
712
713
714
715
716 protected void incTaskCounters(String encodedRegionName) {
717 tasksSent.incrementAndGet();
718
719 AtomicInteger counterPerServer = taskCounterPerRegion.get(encodedRegionName);
720 if (counterPerServer == null) {
721 taskCounterPerRegion.putIfAbsent(encodedRegionName, new AtomicInteger());
722 counterPerServer = taskCounterPerRegion.get(encodedRegionName);
723 }
724 counterPerServer.incrementAndGet();
725 }
726
727
728
729
730 protected void decTaskCounters(String encodedRegionName) {
731 AtomicInteger counterPerServer = taskCounterPerRegion.get(encodedRegionName);
732 counterPerServer.decrementAndGet();
733
734 tasksDone.incrementAndGet();
735 synchronized (tasksDone) {
736 tasksDone.notifyAll();
737 }
738 }
739
740
741
742
743
744
745
746
747 protected HConnectionManager.ServerErrorTracker createServerErrorTracker() {
748 if (useServerTrackerForRetries){
749 return new HConnectionManager.ServerErrorTracker(this.serverTrackerTimeout);
750 }else {
751 return null;
752 }
753 }
754 }