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