1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import java.io.Closeable;
22 import java.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.util.ArrayList;
25 import java.util.Collections;
26 import java.util.LinkedList;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.NavigableMap;
30 import java.util.TreeMap;
31 import java.util.concurrent.Callable;
32 import java.util.concurrent.ExecutionException;
33 import java.util.concurrent.ExecutorService;
34 import java.util.concurrent.Future;
35 import java.util.concurrent.SynchronousQueue;
36 import java.util.concurrent.ThreadPoolExecutor;
37 import java.util.concurrent.TimeUnit;
38
39 import com.google.protobuf.InvalidProtocolBufferException;
40
41 import org.apache.commons.logging.Log;
42 import org.apache.commons.logging.LogFactory;
43 import org.apache.hadoop.hbase.classification.InterfaceAudience;
44 import org.apache.hadoop.hbase.classification.InterfaceStability;
45 import org.apache.hadoop.conf.Configuration;
46 import org.apache.hadoop.hbase.Cell;
47 import org.apache.hadoop.hbase.DoNotRetryIOException;
48 import org.apache.hadoop.hbase.HBaseConfiguration;
49 import org.apache.hadoop.hbase.HConstants;
50 import org.apache.hadoop.hbase.HRegionInfo;
51 import org.apache.hadoop.hbase.HRegionLocation;
52 import org.apache.hadoop.hbase.HTableDescriptor;
53 import org.apache.hadoop.hbase.KeyValue;
54 import org.apache.hadoop.hbase.KeyValueUtil;
55 import org.apache.hadoop.hbase.ServerName;
56 import org.apache.hadoop.hbase.TableName;
57 import org.apache.hadoop.hbase.client.coprocessor.Batch;
58 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
59 import org.apache.hadoop.hbase.filter.BinaryComparator;
60 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
61 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
62 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
63 import org.apache.hadoop.hbase.ipc.RegionCoprocessorRpcChannel;
64 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
65 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
66 import org.apache.hadoop.hbase.protobuf.RequestConverter;
67 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
68 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
69 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateRequest;
70 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutateResponse;
71 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
72 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.CompareType;
73 import org.apache.hadoop.hbase.util.Bytes;
74 import org.apache.hadoop.hbase.util.Pair;
75 import org.apache.hadoop.hbase.util.Threads;
76
77 import com.google.protobuf.Descriptors;
78 import com.google.protobuf.Message;
79 import com.google.protobuf.Service;
80 import com.google.protobuf.ServiceException;
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123 @InterfaceAudience.Public
124 @InterfaceStability.Stable
125 public class HTable implements HTableInterface {
126 private static final Log LOG = LogFactory.getLog(HTable.class);
127 protected HConnection connection;
128 private final TableName tableName;
129 private volatile Configuration configuration;
130 protected List<Row> writeAsyncBuffer = new LinkedList<Row>();
131 private long writeBufferSize;
132 private boolean clearBufferOnFail;
133 private boolean autoFlush;
134 protected long currentWriteBufferSize;
135 protected int scannerCaching;
136 private int maxKeyValueSize;
137 private ExecutorService pool;
138 private boolean closed;
139 private int operationTimeout;
140 private final boolean cleanupPoolOnClose;
141 private final boolean cleanupConnectionOnClose;
142
143
144 protected AsyncProcess<Object> ap;
145 private RpcRetryingCallerFactory rpcCallerFactory;
146 private RpcControllerFactory rpcControllerFactory;
147
148
149
150
151
152
153
154
155
156
157
158 public HTable(Configuration conf, final String tableName)
159 throws IOException {
160 this(conf, TableName.valueOf(tableName));
161 }
162
163
164
165
166
167
168
169
170
171
172
173 public HTable(Configuration conf, final byte[] tableName)
174 throws IOException {
175 this(conf, TableName.valueOf(tableName));
176 }
177
178
179
180
181
182
183
184
185
186
187
188
189
190 public HTable(Configuration conf, final TableName tableName)
191 throws IOException {
192 this.tableName = tableName;
193 this.cleanupPoolOnClose = this.cleanupConnectionOnClose = true;
194 if (conf == null) {
195 this.connection = null;
196 return;
197 }
198 this.connection = HConnectionManager.getConnection(conf);
199 this.configuration = conf;
200
201 this.pool = getDefaultExecutor(conf);
202 this.finishSetup();
203 }
204
205
206
207
208
209
210
211
212
213 public HTable(TableName tableName, HConnection connection) throws IOException {
214 this.tableName = tableName;
215 this.cleanupPoolOnClose = true;
216 this.cleanupConnectionOnClose = false;
217 this.connection = connection;
218 this.configuration = connection.getConfiguration();
219
220 this.pool = getDefaultExecutor(this.configuration);
221 this.finishSetup();
222 }
223
224 public static ThreadPoolExecutor getDefaultExecutor(Configuration conf) {
225 int maxThreads = conf.getInt("hbase.htable.threads.max", Integer.MAX_VALUE);
226 if (maxThreads == 0) {
227 maxThreads = 1;
228 }
229 long keepAliveTime = conf.getLong("hbase.htable.threads.keepalivetime", 60);
230
231
232
233
234
235 ThreadPoolExecutor pool = new ThreadPoolExecutor(1, maxThreads, keepAliveTime, TimeUnit.SECONDS,
236 new SynchronousQueue<Runnable>(), Threads.newDaemonThreadFactory("htable"));
237 ((ThreadPoolExecutor) pool).allowCoreThreadTimeOut(true);
238 return pool;
239 }
240
241
242
243
244
245
246
247
248
249
250
251
252
253 public HTable(Configuration conf, final byte[] tableName, final ExecutorService pool)
254 throws IOException {
255 this(conf, TableName.valueOf(tableName), pool);
256 }
257
258
259
260
261
262
263
264
265
266
267
268
269
270 public HTable(Configuration conf, final TableName tableName, final ExecutorService pool)
271 throws IOException {
272 this.connection = HConnectionManager.getConnection(conf);
273 this.configuration = conf;
274 this.pool = pool;
275 this.tableName = tableName;
276 this.cleanupPoolOnClose = false;
277 this.cleanupConnectionOnClose = true;
278
279 this.finishSetup();
280 }
281
282
283
284
285
286
287
288
289
290
291
292
293 public HTable(final byte[] tableName, final HConnection connection,
294 final ExecutorService pool) throws IOException {
295 this(TableName.valueOf(tableName), connection, pool);
296 }
297
298
299
300
301
302
303
304
305
306
307
308
309 public HTable(TableName tableName, final HConnection connection,
310 final ExecutorService pool) throws IOException {
311 if (connection == null || connection.isClosed()) {
312 throw new IllegalArgumentException("Connection is null or closed.");
313 }
314 this.tableName = tableName;
315 this.cleanupPoolOnClose = this.cleanupConnectionOnClose = false;
316 this.connection = connection;
317 this.configuration = connection.getConfiguration();
318 this.pool = pool;
319
320 this.finishSetup();
321 }
322
323
324
325
326 protected HTable(){
327 tableName = null;
328 cleanupPoolOnClose = false;
329 cleanupConnectionOnClose = false;
330 }
331
332
333
334
335 public static int getMaxKeyValueSize(Configuration conf) {
336 return conf.getInt("hbase.client.keyvalue.maxsize", -1);
337 }
338
339
340
341
342 private void finishSetup() throws IOException {
343 this.operationTimeout = tableName.isSystemTable() ?
344 this.configuration.getInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT,
345 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT):
346 this.configuration.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
347 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
348 this.writeBufferSize = this.configuration.getLong(
349 "hbase.client.write.buffer", 2097152);
350 this.clearBufferOnFail = true;
351 this.autoFlush = true;
352 this.currentWriteBufferSize = 0;
353 this.scannerCaching = this.configuration.getInt(
354 HConstants.HBASE_CLIENT_SCANNER_CACHING,
355 HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
356
357 this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(configuration);
358 this.rpcControllerFactory = RpcControllerFactory.instantiate(configuration);
359 ap = new AsyncProcess<Object>(connection, tableName, pool, null,
360 configuration, rpcCallerFactory, rpcControllerFactory);
361
362 this.maxKeyValueSize = getMaxKeyValueSize(this.configuration);
363 this.closed = false;
364 }
365
366
367
368
369
370
371 @Override
372 public Configuration getConfiguration() {
373 return configuration;
374 }
375
376
377
378
379
380
381
382
383
384
385 @Deprecated
386 public static boolean isTableEnabled(String tableName) throws IOException {
387 return isTableEnabled(TableName.valueOf(tableName));
388 }
389
390
391
392
393
394
395
396
397
398
399 @Deprecated
400 public static boolean isTableEnabled(byte[] tableName) throws IOException {
401 return isTableEnabled(TableName.valueOf(tableName));
402 }
403
404
405
406
407
408
409
410
411
412
413 @Deprecated
414 public static boolean isTableEnabled(TableName tableName) throws IOException {
415 return isTableEnabled(HBaseConfiguration.create(), tableName);
416 }
417
418
419
420
421
422
423
424
425
426 @Deprecated
427 public static boolean isTableEnabled(Configuration conf, String tableName)
428 throws IOException {
429 return isTableEnabled(conf, TableName.valueOf(tableName));
430 }
431
432
433
434
435
436
437
438
439
440 @Deprecated
441 public static boolean isTableEnabled(Configuration conf, byte[] tableName)
442 throws IOException {
443 return isTableEnabled(conf, TableName.valueOf(tableName));
444 }
445
446
447
448
449
450
451
452
453
454 @Deprecated
455 public static boolean isTableEnabled(Configuration conf,
456 final TableName tableName) throws IOException {
457 return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
458 @Override
459 public Boolean connect(HConnection connection) throws IOException {
460 return connection.isTableEnabled(tableName);
461 }
462 });
463 }
464
465
466
467
468
469
470
471 public HRegionLocation getRegionLocation(final String row)
472 throws IOException {
473 return connection.getRegionLocation(tableName, Bytes.toBytes(row), false);
474 }
475
476
477
478
479
480
481
482 public HRegionLocation getRegionLocation(final byte [] row)
483 throws IOException {
484 return connection.getRegionLocation(tableName, row, false);
485 }
486
487
488
489
490
491
492
493
494 public HRegionLocation getRegionLocation(final byte [] row, boolean reload)
495 throws IOException {
496 return connection.getRegionLocation(tableName, row, reload);
497 }
498
499
500
501
502 @Override
503 public byte [] getTableName() {
504 return this.tableName.getName();
505 }
506
507 @Override
508 public TableName getName() {
509 return tableName;
510 }
511
512
513
514
515
516
517
518
519 @Deprecated
520 public HConnection getConnection() {
521 return this.connection;
522 }
523
524
525
526
527
528
529
530 @Deprecated
531 public int getScannerCaching() {
532 return scannerCaching;
533 }
534
535
536
537
538
539 @Deprecated
540 public List<Row> getWriteBuffer() {
541 return writeAsyncBuffer;
542 }
543
544
545
546
547
548
549
550
551
552
553
554
555 @Deprecated
556 public void setScannerCaching(int scannerCaching) {
557 this.scannerCaching = scannerCaching;
558 }
559
560
561
562
563 @Override
564 public HTableDescriptor getTableDescriptor() throws IOException {
565 return new UnmodifyableHTableDescriptor(
566 this.connection.getHTableDescriptor(this.tableName));
567 }
568
569
570
571
572
573
574
575
576 public byte [][] getStartKeys() throws IOException {
577 return getStartEndKeys().getFirst();
578 }
579
580
581
582
583
584
585
586
587 public byte[][] getEndKeys() throws IOException {
588 return getStartEndKeys().getSecond();
589 }
590
591
592
593
594
595
596
597
598
599 public Pair<byte[][],byte[][]> getStartEndKeys() throws IOException {
600 NavigableMap<HRegionInfo, ServerName> regions = getRegionLocations();
601 final List<byte[]> startKeyList = new ArrayList<byte[]>(regions.size());
602 final List<byte[]> endKeyList = new ArrayList<byte[]>(regions.size());
603
604 for (HRegionInfo region : regions.keySet()) {
605 startKeyList.add(region.getStartKey());
606 endKeyList.add(region.getEndKey());
607 }
608
609 return new Pair<byte [][], byte [][]>(
610 startKeyList.toArray(new byte[startKeyList.size()][]),
611 endKeyList.toArray(new byte[endKeyList.size()][]));
612 }
613
614
615
616
617
618
619
620
621 public NavigableMap<HRegionInfo, ServerName> getRegionLocations() throws IOException {
622
623 return MetaScanner.allTableRegions(getConfiguration(), this.connection, getName(), false);
624 }
625
626
627
628
629
630
631
632
633
634
635 public List<HRegionLocation> getRegionsInRange(final byte [] startKey,
636 final byte [] endKey) throws IOException {
637 return getRegionsInRange(startKey, endKey, false);
638 }
639
640
641
642
643
644
645
646
647
648
649
650 public List<HRegionLocation> getRegionsInRange(final byte [] startKey,
651 final byte [] endKey, final boolean reload) throws IOException {
652 return getKeysAndRegionsInRange(startKey, endKey, false, reload).getSecond();
653 }
654
655
656
657
658
659
660
661
662
663
664
665
666 private Pair<List<byte[]>, List<HRegionLocation>> getKeysAndRegionsInRange(
667 final byte[] startKey, final byte[] endKey, final boolean includeEndKey)
668 throws IOException {
669 return getKeysAndRegionsInRange(startKey, endKey, includeEndKey, false);
670 }
671
672
673
674
675
676
677
678
679
680
681
682
683
684 private Pair<List<byte[]>, List<HRegionLocation>> getKeysAndRegionsInRange(
685 final byte[] startKey, final byte[] endKey, final boolean includeEndKey,
686 final boolean reload) throws IOException {
687 final boolean endKeyIsEndOfTable = Bytes.equals(endKey,HConstants.EMPTY_END_ROW);
688 if ((Bytes.compareTo(startKey, endKey) > 0) && !endKeyIsEndOfTable) {
689 throw new IllegalArgumentException(
690 "Invalid range: " + Bytes.toStringBinary(startKey) +
691 " > " + Bytes.toStringBinary(endKey));
692 }
693 List<byte[]> keysInRange = new ArrayList<byte[]>();
694 List<HRegionLocation> regionsInRange = new ArrayList<HRegionLocation>();
695 byte[] currentKey = startKey;
696 do {
697 HRegionLocation regionLocation = getRegionLocation(currentKey, reload);
698 keysInRange.add(currentKey);
699 regionsInRange.add(regionLocation);
700 currentKey = regionLocation.getRegionInfo().getEndKey();
701 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW)
702 && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0
703 || (includeEndKey && Bytes.compareTo(currentKey, endKey) == 0)));
704 return new Pair<List<byte[]>, List<HRegionLocation>>(keysInRange,
705 regionsInRange);
706 }
707
708
709
710
711 @Override
712 public Result getRowOrBefore(final byte[] row, final byte[] family)
713 throws IOException {
714 RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
715 tableName, row) {
716 public Result call() throws IOException {
717 return ProtobufUtil.getRowOrBefore(getStub(), getLocation().getRegionInfo()
718 .getRegionName(), row, family, rpcControllerFactory.newController());
719 }
720 };
721 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
722 }
723
724
725
726
727 @Override
728 public ResultScanner getScanner(final Scan scan) throws IOException {
729 if (scan.getCaching() <= 0) {
730 scan.setCaching(getScannerCaching());
731 }
732
733 if (scan.isReversed()) {
734 if (scan.isSmall()) {
735 return new ClientSmallReversedScanner(getConfiguration(), scan, getName(),
736 this.connection);
737 } else {
738 return new ReversedClientScanner(getConfiguration(), scan, getName(), this.connection);
739 }
740 }
741
742 if (scan.isSmall()) {
743 return new ClientSmallScanner(getConfiguration(), scan, getName(), this.connection);
744 } else {
745 return new ClientScanner(getConfiguration(), scan, getName(), this.connection);
746 }
747 }
748
749
750
751
752 @Override
753 public ResultScanner getScanner(byte [] family) throws IOException {
754 Scan scan = new Scan();
755 scan.addFamily(family);
756 return getScanner(scan);
757 }
758
759
760
761
762 @Override
763 public ResultScanner getScanner(byte [] family, byte [] qualifier)
764 throws IOException {
765 Scan scan = new Scan();
766 scan.addColumn(family, qualifier);
767 return getScanner(scan);
768 }
769
770
771
772
773 @Override
774 public Result get(final Get get) throws IOException {
775
776
777
778 final PayloadCarryingRpcController controller = rpcControllerFactory.newController();
779 controller.setPriority(tableName);
780 RegionServerCallable<Result> callable =
781 new RegionServerCallable<Result>(this.connection, getName(), get.getRow()) {
782 public Result call() throws IOException {
783 return ProtobufUtil.get(getStub(), getLocation().getRegionInfo().getRegionName(), get,
784 controller);
785 }
786 };
787 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
788 }
789
790
791
792
793 @Override
794 public Result[] get(List<Get> gets) throws IOException {
795 if (gets.size() == 1) {
796 return new Result[]{get(gets.get(0))};
797 }
798 try {
799 Object [] r1 = batch((List)gets);
800
801
802 Result [] results = new Result[r1.length];
803 int i=0;
804 for (Object o : r1) {
805
806 results[i++] = (Result) o;
807 }
808
809 return results;
810 } catch (InterruptedException e) {
811 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
812 }
813 }
814
815
816
817
818 @Override
819 public void batch(final List<?extends Row> actions, final Object[] results)
820 throws InterruptedException, IOException {
821 batchCallback(actions, results, null);
822 }
823
824
825
826
827
828
829 @Override
830 public Object[] batch(final List<? extends Row> actions)
831 throws InterruptedException, IOException {
832 return batchCallback(actions, null);
833 }
834
835
836
837
838 @Override
839 public <R> void batchCallback(
840 final List<? extends Row> actions, final Object[] results, final Batch.Callback<R> callback)
841 throws IOException, InterruptedException {
842 connection.processBatchCallback(actions, tableName, pool, results, callback);
843 }
844
845
846
847
848
849
850
851
852 @Override
853 public <R> Object[] batchCallback(
854 final List<? extends Row> actions, final Batch.Callback<R> callback) throws IOException,
855 InterruptedException {
856 Object[] results = new Object[actions.size()];
857 batchCallback(actions, results, callback);
858 return results;
859 }
860
861
862
863
864 @Override
865 public void delete(final Delete delete)
866 throws IOException {
867 RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(connection,
868 tableName, delete.getRow()) {
869 public Boolean call() throws IOException {
870 try {
871 MutateRequest request = RequestConverter.buildMutateRequest(
872 getLocation().getRegionInfo().getRegionName(), delete);
873 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
874 controller.setPriority(tableName);
875 MutateResponse response = getStub().mutate(controller, request);
876 return Boolean.valueOf(response.getProcessed());
877 } catch (ServiceException se) {
878 throw ProtobufUtil.getRemoteException(se);
879 }
880 }
881 };
882 rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
883 }
884
885
886
887
888 @Override
889 public void delete(final List<Delete> deletes)
890 throws IOException {
891 Object[] results = new Object[deletes.size()];
892 try {
893 batch(deletes, results);
894 } catch (InterruptedException e) {
895 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
896 } finally {
897
898
899
900 for (int i = results.length - 1; i>=0; i--) {
901
902 if (results[i] instanceof Result) {
903 deletes.remove(i);
904 }
905 }
906 }
907 }
908
909
910
911
912 @Override
913 public void put(final Put put)
914 throws InterruptedIOException, RetriesExhaustedWithDetailsException {
915 doPut(put);
916 if (autoFlush) {
917 flushCommits();
918 }
919 }
920
921
922
923
924 @Override
925 public void put(final List<Put> puts)
926 throws InterruptedIOException, RetriesExhaustedWithDetailsException {
927 for (Put put : puts) {
928 doPut(put);
929 }
930 if (autoFlush) {
931 flushCommits();
932 }
933 }
934
935
936
937
938
939
940
941
942 private void doPut(Put put) throws InterruptedIOException, RetriesExhaustedWithDetailsException {
943 if (ap.hasError()){
944 writeAsyncBuffer.add(put);
945 backgroundFlushCommits(true);
946 }
947
948 validatePut(put);
949
950 currentWriteBufferSize += put.heapSize();
951 writeAsyncBuffer.add(put);
952
953 while (currentWriteBufferSize > writeBufferSize) {
954 backgroundFlushCommits(false);
955 }
956 }
957
958
959
960
961
962
963
964
965
966 private void backgroundFlushCommits(boolean synchronous) throws
967 InterruptedIOException, RetriesExhaustedWithDetailsException {
968
969 try {
970 do {
971 ap.submit(writeAsyncBuffer, true);
972 } while (synchronous && !writeAsyncBuffer.isEmpty());
973
974 if (synchronous) {
975 ap.waitUntilDone();
976 }
977
978 if (ap.hasError()) {
979 LOG.debug(tableName + ": One or more of the operations have failed -" +
980 " waiting for all operation in progress to finish (successfully or not)");
981 while (!writeAsyncBuffer.isEmpty()) {
982 ap.submit(writeAsyncBuffer, true);
983 }
984 ap.waitUntilDone();
985
986 if (!clearBufferOnFail) {
987
988
989 writeAsyncBuffer.addAll(ap.getFailedOperations());
990 }
991 RetriesExhaustedWithDetailsException e = ap.getErrors();
992 ap.clearErrors();
993 throw e;
994 }
995 } finally {
996 currentWriteBufferSize = 0;
997 for (Row mut : writeAsyncBuffer) {
998 if (mut instanceof Mutation) {
999 currentWriteBufferSize += ((Mutation) mut).heapSize();
1000 }
1001 }
1002 }
1003 }
1004
1005
1006
1007
1008 @Override
1009 public void mutateRow(final RowMutations rm) throws IOException {
1010 RegionServerCallable<Void> callable =
1011 new RegionServerCallable<Void>(connection, getName(), rm.getRow()) {
1012 public Void call() throws IOException {
1013 try {
1014 RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
1015 getLocation().getRegionInfo().getRegionName(), rm);
1016 regionMutationBuilder.setAtomic(true);
1017 MultiRequest request =
1018 MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
1019 PayloadCarryingRpcController pcrc = rpcControllerFactory.newController();
1020 pcrc.setPriority(tableName);
1021 getStub().multi(pcrc, request);
1022 } catch (ServiceException se) {
1023 throw ProtobufUtil.getRemoteException(se);
1024 }
1025 return null;
1026 }
1027 };
1028 rpcCallerFactory.<Void> newCaller().callWithRetries(callable, this.operationTimeout);
1029 }
1030
1031
1032
1033
1034 @Override
1035 public Result append(final Append append) throws IOException {
1036 if (append.numFamilies() == 0) {
1037 throw new IOException(
1038 "Invalid arguments to append, no columns specified");
1039 }
1040
1041 NonceGenerator ng = this.connection.getNonceGenerator();
1042 final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
1043 RegionServerCallable<Result> callable =
1044 new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
1045 public Result call() throws IOException {
1046 try {
1047 MutateRequest request = RequestConverter.buildMutateRequest(
1048 getLocation().getRegionInfo().getRegionName(), append, nonceGroup, nonce);
1049 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1050 rpcController.setPriority(getTableName());
1051 MutateResponse response = getStub().mutate(rpcController, request);
1052 if (!response.hasResult()) return null;
1053 return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1054 } catch (ServiceException se) {
1055 throw ProtobufUtil.getRemoteException(se);
1056 }
1057 }
1058 };
1059 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
1060 }
1061
1062
1063
1064
1065 @Override
1066 public Result increment(final Increment increment) throws IOException {
1067 if (!increment.hasFamilies()) {
1068 throw new IOException(
1069 "Invalid arguments to increment, no columns specified");
1070 }
1071 NonceGenerator ng = this.connection.getNonceGenerator();
1072 final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
1073 RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
1074 getName(), increment.getRow()) {
1075 public Result call() throws IOException {
1076 try {
1077 MutateRequest request = RequestConverter.buildMutateRequest(
1078 getLocation().getRegionInfo().getRegionName(), increment, nonceGroup, nonce);
1079 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1080 rpcController.setPriority(getTableName());
1081 MutateResponse response = getStub().mutate(rpcController, request);
1082 return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1083 } catch (ServiceException se) {
1084 throw ProtobufUtil.getRemoteException(se);
1085 }
1086 }
1087 };
1088 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
1089 }
1090
1091
1092
1093
1094 @Override
1095 public long incrementColumnValue(final byte [] row, final byte [] family,
1096 final byte [] qualifier, final long amount)
1097 throws IOException {
1098 return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
1099 }
1100
1101
1102
1103
1104 @Deprecated
1105 @Override
1106 public long incrementColumnValue(final byte [] row, final byte [] family,
1107 final byte [] qualifier, final long amount, final boolean writeToWAL)
1108 throws IOException {
1109 return incrementColumnValue(row, family, qualifier, amount,
1110 writeToWAL? Durability.SKIP_WAL: Durability.USE_DEFAULT);
1111 }
1112
1113
1114
1115
1116 @Override
1117 public long incrementColumnValue(final byte [] row, final byte [] family,
1118 final byte [] qualifier, final long amount, final Durability durability)
1119 throws IOException {
1120 NullPointerException npe = null;
1121 if (row == null) {
1122 npe = new NullPointerException("row is null");
1123 } else if (family == null) {
1124 npe = new NullPointerException("family is null");
1125 } else if (qualifier == null) {
1126 npe = new NullPointerException("qualifier is null");
1127 }
1128 if (npe != null) {
1129 throw new IOException(
1130 "Invalid arguments to incrementColumnValue", npe);
1131 }
1132
1133 NonceGenerator ng = this.connection.getNonceGenerator();
1134 final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
1135 RegionServerCallable<Long> callable =
1136 new RegionServerCallable<Long>(connection, getName(), row) {
1137 public Long call() throws IOException {
1138 try {
1139 MutateRequest request = RequestConverter.buildIncrementRequest(
1140 getLocation().getRegionInfo().getRegionName(), row, family,
1141 qualifier, amount, durability, nonceGroup, nonce);
1142 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1143 rpcController.setPriority(getTableName());
1144 MutateResponse response = getStub().mutate(rpcController, request);
1145 Result result =
1146 ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1147 return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
1148 } catch (ServiceException se) {
1149 throw ProtobufUtil.getRemoteException(se);
1150 }
1151 }
1152 };
1153 return rpcCallerFactory.<Long> newCaller().callWithRetries(callable, this.operationTimeout);
1154 }
1155
1156
1157
1158
1159 @Override
1160 public boolean checkAndPut(final byte [] row,
1161 final byte [] family, final byte [] qualifier, final byte [] value,
1162 final Put put)
1163 throws IOException {
1164 RegionServerCallable<Boolean> callable =
1165 new RegionServerCallable<Boolean>(connection, getName(), row) {
1166 public Boolean call() throws IOException {
1167 try {
1168 MutateRequest request = RequestConverter.buildMutateRequest(
1169 getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
1170 new BinaryComparator(value), CompareType.EQUAL, put);
1171 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1172 rpcController.setPriority(getTableName());
1173 MutateResponse response = getStub().mutate(rpcController, request);
1174 return Boolean.valueOf(response.getProcessed());
1175 } catch (ServiceException se) {
1176 throw ProtobufUtil.getRemoteException(se);
1177 }
1178 }
1179 };
1180 return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
1181 }
1182
1183
1184
1185
1186
1187 @Override
1188 public boolean checkAndDelete(final byte [] row,
1189 final byte [] family, final byte [] qualifier, final byte [] value,
1190 final Delete delete)
1191 throws IOException {
1192 RegionServerCallable<Boolean> callable =
1193 new RegionServerCallable<Boolean>(connection, getName(), row) {
1194 public Boolean call() throws IOException {
1195 try {
1196 MutateRequest request = RequestConverter.buildMutateRequest(
1197 getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
1198 new BinaryComparator(value), CompareType.EQUAL, delete);
1199 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1200 rpcController.setPriority(getTableName());
1201 MutateResponse response = getStub().mutate(rpcController, request);
1202 return Boolean.valueOf(response.getProcessed());
1203 } catch (ServiceException se) {
1204 throw ProtobufUtil.getRemoteException(se);
1205 }
1206 }
1207 };
1208 return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
1209 }
1210
1211
1212
1213
1214 @Override
1215 public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
1216 final CompareOp compareOp, final byte [] value, final RowMutations rm)
1217 throws IOException {
1218 RegionServerCallable<Boolean> callable =
1219 new RegionServerCallable<Boolean>(connection, getName(), row) {
1220 @Override
1221 public Boolean call() throws IOException {
1222 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1223 controller.setPriority(tableName);
1224 try {
1225 CompareType compareType = CompareType.valueOf(compareOp.name());
1226 MultiRequest request = RequestConverter.buildMutateRequest(
1227 getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
1228 new BinaryComparator(value), compareType, rm);
1229 ClientProtos.MultiResponse response = getStub().multi(controller, request);
1230 return Boolean.valueOf(response.getProcessed());
1231 } catch (ServiceException se) {
1232 throw ProtobufUtil.getRemoteException(se);
1233 }
1234 }
1235 };
1236 return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
1237 }
1238
1239
1240
1241
1242 @Override
1243 public boolean exists(final Get get) throws IOException {
1244 get.setCheckExistenceOnly(true);
1245 Result r = get(get);
1246 assert r.getExists() != null;
1247 return r.getExists();
1248 }
1249
1250
1251
1252
1253 @Override
1254 public Boolean[] exists(final List<Get> gets) throws IOException {
1255 if (gets.isEmpty()) return new Boolean[]{};
1256 if (gets.size() == 1) return new Boolean[]{exists(gets.get(0))};
1257
1258 for (Get g: gets){
1259 g.setCheckExistenceOnly(true);
1260 }
1261
1262 Object[] r1;
1263 try {
1264 r1 = batch(gets);
1265 } catch (InterruptedException e) {
1266 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
1267 }
1268
1269
1270 Boolean[] results = new Boolean[r1.length];
1271 int i = 0;
1272 for (Object o : r1) {
1273
1274 results[i++] = ((Result)o).getExists();
1275 }
1276
1277 return results;
1278 }
1279
1280
1281
1282
1283 @Override
1284 public void flushCommits() throws InterruptedIOException, RetriesExhaustedWithDetailsException {
1285
1286
1287 backgroundFlushCommits(true);
1288 }
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301 public <R> void processBatchCallback(
1302 final List<? extends Row> list, final Object[] results, final Batch.Callback<R> callback)
1303 throws IOException, InterruptedException {
1304 this.batchCallback(list, results, callback);
1305 }
1306
1307
1308
1309
1310
1311
1312 public void processBatch(final List<? extends Row> list, final Object[] results)
1313 throws IOException, InterruptedException {
1314
1315 this.processBatchCallback(list, results, null);
1316 }
1317
1318
1319 @Override
1320 public void close() throws IOException {
1321 if (this.closed) {
1322 return;
1323 }
1324 flushCommits();
1325 if (cleanupPoolOnClose) {
1326 this.pool.shutdown();
1327 }
1328 if (cleanupConnectionOnClose) {
1329 if (this.connection != null) {
1330 this.connection.close();
1331 }
1332 }
1333 this.closed = true;
1334 }
1335
1336
1337 public void validatePut(final Put put) throws IllegalArgumentException {
1338 validatePut(put, maxKeyValueSize);
1339 }
1340
1341
1342 public static void validatePut(Put put, int maxKeyValueSize) throws IllegalArgumentException {
1343 if (put.isEmpty()) {
1344 throw new IllegalArgumentException("No columns to insert");
1345 }
1346 if (maxKeyValueSize > 0) {
1347 for (List<Cell> list : put.getFamilyCellMap().values()) {
1348 for (Cell cell : list) {
1349
1350 KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
1351 if (kv.getLength() > maxKeyValueSize) {
1352 throw new IllegalArgumentException("KeyValue size too large");
1353 }
1354 }
1355 }
1356 }
1357 }
1358
1359
1360
1361
1362 @Override
1363 public boolean isAutoFlush() {
1364 return autoFlush;
1365 }
1366
1367
1368
1369
1370 @Deprecated
1371 @Override
1372 public void setAutoFlush(boolean autoFlush) {
1373 setAutoFlush(autoFlush, autoFlush);
1374 }
1375
1376
1377
1378
1379 @Override
1380 public void setAutoFlushTo(boolean autoFlush) {
1381 setAutoFlush(autoFlush, clearBufferOnFail);
1382 }
1383
1384
1385
1386
1387 @Override
1388 public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
1389 this.autoFlush = autoFlush;
1390 this.clearBufferOnFail = autoFlush || clearBufferOnFail;
1391 }
1392
1393
1394
1395
1396
1397
1398
1399
1400 @Override
1401 public long getWriteBufferSize() {
1402 return writeBufferSize;
1403 }
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413 public void setWriteBufferSize(long writeBufferSize) throws IOException {
1414 this.writeBufferSize = writeBufferSize;
1415 if(currentWriteBufferSize > writeBufferSize) {
1416 flushCommits();
1417 }
1418 }
1419
1420
1421
1422
1423
1424 ExecutorService getPool() {
1425 return this.pool;
1426 }
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437 public static void setRegionCachePrefetch(final byte[] tableName,
1438 final boolean enable) throws IOException {
1439 setRegionCachePrefetch(TableName.valueOf(tableName), enable);
1440 }
1441
1442 public static void setRegionCachePrefetch(
1443 final TableName tableName,
1444 final boolean enable) throws IOException {
1445 HConnectionManager.execute(new HConnectable<Void>(HBaseConfiguration.create()) {
1446 @Override
1447 public Void connect(HConnection connection) throws IOException {
1448 connection.setRegionCachePrefetch(tableName, enable);
1449 return null;
1450 }
1451 });
1452 }
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464 public static void setRegionCachePrefetch(final Configuration conf,
1465 final byte[] tableName, final boolean enable) throws IOException {
1466 setRegionCachePrefetch(conf, TableName.valueOf(tableName), enable);
1467 }
1468
1469 public static void setRegionCachePrefetch(final Configuration conf,
1470 final TableName tableName,
1471 final boolean enable) throws IOException {
1472 HConnectionManager.execute(new HConnectable<Void>(conf) {
1473 @Override
1474 public Void connect(HConnection connection) throws IOException {
1475 connection.setRegionCachePrefetch(tableName, enable);
1476 return null;
1477 }
1478 });
1479 }
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489 public static boolean getRegionCachePrefetch(final Configuration conf,
1490 final byte[] tableName) throws IOException {
1491 return getRegionCachePrefetch(conf, TableName.valueOf(tableName));
1492 }
1493
1494 public static boolean getRegionCachePrefetch(final Configuration conf,
1495 final TableName tableName) throws IOException {
1496 return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
1497 @Override
1498 public Boolean connect(HConnection connection) throws IOException {
1499 return connection.getRegionCachePrefetch(tableName);
1500 }
1501 });
1502 }
1503
1504
1505
1506
1507
1508
1509
1510
1511 public static boolean getRegionCachePrefetch(final byte[] tableName) throws IOException {
1512 return getRegionCachePrefetch(TableName.valueOf(tableName));
1513 }
1514
1515 public static boolean getRegionCachePrefetch(
1516 final TableName tableName) throws IOException {
1517 return HConnectionManager.execute(new HConnectable<Boolean>(
1518 HBaseConfiguration.create()) {
1519 @Override
1520 public Boolean connect(HConnection connection) throws IOException {
1521 return connection.getRegionCachePrefetch(tableName);
1522 }
1523 });
1524 }
1525
1526
1527
1528
1529
1530 public void clearRegionCache() {
1531 this.connection.clearRegionCache();
1532 }
1533
1534
1535
1536
1537 public CoprocessorRpcChannel coprocessorService(byte[] row) {
1538 return new RegionCoprocessorRpcChannel(connection, tableName, row, rpcCallerFactory,
1539 rpcControllerFactory);
1540 }
1541
1542
1543
1544
1545 @Override
1546 public <T extends Service, R> Map<byte[],R> coprocessorService(final Class<T> service,
1547 byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable)
1548 throws ServiceException, Throwable {
1549 final Map<byte[],R> results = Collections.synchronizedMap(
1550 new TreeMap<byte[], R>(Bytes.BYTES_COMPARATOR));
1551 coprocessorService(service, startKey, endKey, callable, new Batch.Callback<R>() {
1552 public void update(byte[] region, byte[] row, R value) {
1553 if (region != null) {
1554 results.put(region, value);
1555 }
1556 }
1557 });
1558 return results;
1559 }
1560
1561
1562
1563
1564 @Override
1565 public <T extends Service, R> void coprocessorService(final Class<T> service,
1566 byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable,
1567 final Batch.Callback<R> callback) throws ServiceException, Throwable {
1568
1569
1570 List<byte[]> keys = getStartKeysInRange(startKey, endKey);
1571
1572 Map<byte[],Future<R>> futures =
1573 new TreeMap<byte[],Future<R>>(Bytes.BYTES_COMPARATOR);
1574 for (final byte[] r : keys) {
1575 final RegionCoprocessorRpcChannel channel =
1576 new RegionCoprocessorRpcChannel(connection, tableName, r, rpcCallerFactory,
1577 rpcControllerFactory);
1578 Future<R> future = pool.submit(
1579 new Callable<R>() {
1580 public R call() throws Exception {
1581 T instance = ProtobufUtil.newServiceStub(service, channel);
1582 R result = callable.call(instance);
1583 byte[] region = channel.getLastRegion();
1584 if (callback != null) {
1585 callback.update(region, r, result);
1586 }
1587 return result;
1588 }
1589 });
1590 futures.put(r, future);
1591 }
1592 for (Map.Entry<byte[],Future<R>> e : futures.entrySet()) {
1593 try {
1594 e.getValue().get();
1595 } catch (ExecutionException ee) {
1596 LOG.warn("Error calling coprocessor service " + service.getName() + " for row "
1597 + Bytes.toStringBinary(e.getKey()), ee);
1598 throw ee.getCause();
1599 } catch (InterruptedException ie) {
1600 throw new InterruptedIOException("Interrupted calling coprocessor service " + service.getName()
1601 + " for row " + Bytes.toStringBinary(e.getKey()))
1602 .initCause(ie);
1603 }
1604 }
1605 }
1606
1607 private List<byte[]> getStartKeysInRange(byte[] start, byte[] end)
1608 throws IOException {
1609 if (start == null) {
1610 start = HConstants.EMPTY_START_ROW;
1611 }
1612 if (end == null) {
1613 end = HConstants.EMPTY_END_ROW;
1614 }
1615 return getKeysAndRegionsInRange(start, end, true).getFirst();
1616 }
1617
1618 public void setOperationTimeout(int operationTimeout) {
1619 this.operationTimeout = operationTimeout;
1620 }
1621
1622 public int getOperationTimeout() {
1623 return operationTimeout;
1624 }
1625
1626 @Override
1627 public String toString() {
1628 return tableName + ";" + connection;
1629 }
1630
1631
1632
1633
1634
1635
1636 public static void main(String[] args) throws IOException {
1637 HTable t = new HTable(HBaseConfiguration.create(), args[0]);
1638 try {
1639 System.out.println(t.get(new Get(Bytes.toBytes(args[1]))));
1640 } finally {
1641 t.close();
1642 }
1643 }
1644
1645
1646
1647
1648 @Override
1649 public <R extends Message> Map<byte[], R> batchCoprocessorService(
1650 Descriptors.MethodDescriptor methodDescriptor, Message request,
1651 byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable {
1652 final Map<byte[], R> results = Collections.synchronizedMap(new TreeMap<byte[], R>(
1653 Bytes.BYTES_COMPARATOR));
1654 batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
1655 new Callback<R>() {
1656
1657 @Override
1658 public void update(byte[] region, byte[] row, R result) {
1659 if (region != null) {
1660 results.put(region, result);
1661 }
1662 }
1663 });
1664 return results;
1665 }
1666
1667
1668
1669
1670 @Override
1671 public <R extends Message> void batchCoprocessorService(
1672 final Descriptors.MethodDescriptor methodDescriptor, final Message request,
1673 byte[] startKey, byte[] endKey, final R responsePrototype, final Callback<R> callback)
1674 throws ServiceException, Throwable {
1675
1676
1677 Pair<List<byte[]>, List<HRegionLocation>> keysAndRegions =
1678 getKeysAndRegionsInRange(startKey, endKey, true);
1679 List<byte[]> keys = keysAndRegions.getFirst();
1680 List<HRegionLocation> regions = keysAndRegions.getSecond();
1681
1682
1683 if (keys.isEmpty()) {
1684 LOG.info("No regions were selected by key range start=" + Bytes.toStringBinary(startKey) +
1685 ", end=" + Bytes.toStringBinary(endKey));
1686 return;
1687 }
1688
1689 List<RegionCoprocessorServiceExec> execs = new ArrayList<RegionCoprocessorServiceExec>();
1690 final Map<byte[], RegionCoprocessorServiceExec> execsByRow =
1691 new TreeMap<byte[], RegionCoprocessorServiceExec>(Bytes.BYTES_COMPARATOR);
1692 for (int i = 0; i < keys.size(); i++) {
1693 final byte[] rowKey = keys.get(i);
1694 final byte[] region = regions.get(i).getRegionInfo().getRegionName();
1695 RegionCoprocessorServiceExec exec =
1696 new RegionCoprocessorServiceExec(region, rowKey, methodDescriptor, request);
1697 execs.add(exec);
1698 execsByRow.put(rowKey, exec);
1699 }
1700
1701
1702
1703 final List<Throwable> callbackErrorExceptions = new ArrayList<Throwable>();
1704 final List<Row> callbackErrorActions = new ArrayList<Row>();
1705 final List<String> callbackErrorServers = new ArrayList<String>();
1706
1707 AsyncProcess<ClientProtos.CoprocessorServiceResult> asyncProcess =
1708 new AsyncProcess<ClientProtos.CoprocessorServiceResult>(connection, tableName, pool,
1709 new AsyncProcess.AsyncProcessCallback<ClientProtos.CoprocessorServiceResult>() {
1710 @SuppressWarnings("unchecked")
1711 @Override
1712 public void success(int originalIndex, byte[] region, Row row,
1713 ClientProtos.CoprocessorServiceResult serviceResult) {
1714 if (LOG.isTraceEnabled()) {
1715 LOG.trace("Received result for endpoint " + methodDescriptor.getFullName() +
1716 " call #" + originalIndex + ": region=" + Bytes.toStringBinary(region) +
1717 ", row=" + Bytes.toStringBinary(row.getRow()) +
1718 ", value=" + serviceResult.getValue().getValue());
1719 }
1720 try {
1721 callback.update(region, row.getRow(),
1722 (R) responsePrototype.newBuilderForType().mergeFrom(
1723 serviceResult.getValue().getValue()).build());
1724 } catch (InvalidProtocolBufferException e) {
1725 LOG.error("Unexpected response type from endpoint " + methodDescriptor.getFullName(),
1726 e);
1727 callbackErrorExceptions.add(e);
1728 callbackErrorActions.add(row);
1729 callbackErrorServers.add("null");
1730 }
1731 }
1732
1733 @Override
1734 public boolean failure(int originalIndex, byte[] region, Row row, Throwable t) {
1735 RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
1736 LOG.error("Failed calling endpoint " + methodDescriptor.getFullName() + ": region="
1737 + Bytes.toStringBinary(exec.getRegion()), t);
1738 return true;
1739 }
1740
1741 @Override
1742 public boolean retriableFailure(int originalIndex, Row row, byte[] region,
1743 Throwable exception) {
1744 RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
1745 LOG.error("Failed calling endpoint " + methodDescriptor.getFullName() + ": region="
1746 + Bytes.toStringBinary(exec.getRegion()), exception);
1747 return !(exception instanceof DoNotRetryIOException);
1748 }
1749 },
1750 configuration, rpcCallerFactory, rpcControllerFactory);
1751
1752 asyncProcess.submitAll(execs);
1753 asyncProcess.waitUntilDone();
1754
1755 if (asyncProcess.hasError()) {
1756 throw asyncProcess.getErrors();
1757 } else if (!callbackErrorExceptions.isEmpty()) {
1758 throw new RetriesExhaustedWithDetailsException(callbackErrorExceptions, callbackErrorActions,
1759 callbackErrorServers);
1760 }
1761 }
1762 }