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