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