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.RpcClient;
63 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
64 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
65 import org.apache.hadoop.hbase.protobuf.RequestConverter;
66 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
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 final RetryingTimeTracker tracker = new RetryingTimeTracker();
1059 PayloadCarryingServerCallable<MultiResponse> callable =
1060 new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
1061 rpcControllerFactory) {
1062 @Override
1063 public MultiResponse call() throws IOException {
1064 tracker.start();
1065 controller.setPriority(tableName);
1066 int remainingTime = tracker.getRemainingTime(operationTimeout);
1067 if (remainingTime == 0) {
1068 throw new DoNotRetryIOException("Timeout for mutate row");
1069 }
1070 RpcClient.setRpcTimeout(remainingTime);
1071 try {
1072 RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
1073 getLocation().getRegionInfo().getRegionName(), rm);
1074 regionMutationBuilder.setAtomic(true);
1075 MultiRequest request =
1076 MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
1077 ClientProtos.MultiResponse response = getStub().multi(controller, request);
1078 ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
1079 if (res.hasException()) {
1080 Throwable ex = ProtobufUtil.toException(res.getException());
1081 if (ex instanceof IOException) {
1082 throw (IOException) ex;
1083 }
1084 throw new IOException("Failed to mutate row: " +
1085 Bytes.toStringBinary(rm.getRow()), ex);
1086 }
1087 return ResponseConverter.getResults(request, response, controller.cellScanner());
1088 } catch (ServiceException se) {
1089 throw ProtobufUtil.getRemoteException(se);
1090 }
1091 }
1092 };
1093 ap.submitAll(rm.getMutations(), null, callable);
1094 ap.waitUntilDone();
1095 if (ap.hasError()) {
1096 throw ap.getErrors();
1097 }
1098 }
1099
1100
1101
1102
1103 @Override
1104 public Result append(final Append append) throws IOException {
1105 if (append.numFamilies() == 0) {
1106 throw new IOException(
1107 "Invalid arguments to append, no columns specified");
1108 }
1109
1110 NonceGenerator ng = this.connection.getNonceGenerator();
1111 final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
1112 RegionServerCallable<Result> callable =
1113 new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
1114 public Result call() throws IOException {
1115 try {
1116 MutateRequest request = RequestConverter.buildMutateRequest(
1117 getLocation().getRegionInfo().getRegionName(), append, nonceGroup, nonce);
1118 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1119 rpcController.setPriority(getTableName());
1120 MutateResponse response = getStub().mutate(rpcController, request);
1121 if (!response.hasResult()) return null;
1122 return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1123 } catch (ServiceException se) {
1124 throw ProtobufUtil.getRemoteException(se);
1125 }
1126 }
1127 };
1128 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
1129 }
1130
1131
1132
1133
1134 @Override
1135 public Result increment(final Increment increment) throws IOException {
1136 if (!increment.hasFamilies()) {
1137 throw new IOException(
1138 "Invalid arguments to increment, no columns specified");
1139 }
1140 NonceGenerator ng = this.connection.getNonceGenerator();
1141 final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
1142 RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
1143 getName(), increment.getRow()) {
1144 public Result call() throws IOException {
1145 try {
1146 MutateRequest request = RequestConverter.buildMutateRequest(
1147 getLocation().getRegionInfo().getRegionName(), increment, nonceGroup, nonce);
1148 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1149 rpcController.setPriority(getTableName());
1150 MutateResponse response = getStub().mutate(rpcController, request);
1151 return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1152 } catch (ServiceException se) {
1153 throw ProtobufUtil.getRemoteException(se);
1154 }
1155 }
1156 };
1157 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
1158 }
1159
1160
1161
1162
1163 @Override
1164 public long incrementColumnValue(final byte [] row, final byte [] family,
1165 final byte [] qualifier, final long amount)
1166 throws IOException {
1167 return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
1168 }
1169
1170
1171
1172
1173 @Deprecated
1174 @Override
1175 public long incrementColumnValue(final byte [] row, final byte [] family,
1176 final byte [] qualifier, final long amount, final boolean writeToWAL)
1177 throws IOException {
1178 return incrementColumnValue(row, family, qualifier, amount,
1179 writeToWAL? Durability.SYNC_WAL: Durability.SKIP_WAL);
1180 }
1181
1182
1183
1184
1185 @Override
1186 public long incrementColumnValue(final byte [] row, final byte [] family,
1187 final byte [] qualifier, final long amount, final Durability durability)
1188 throws IOException {
1189 NullPointerException npe = null;
1190 if (row == null) {
1191 npe = new NullPointerException("row is null");
1192 } else if (family == null) {
1193 npe = new NullPointerException("family is null");
1194 } else if (qualifier == null) {
1195 npe = new NullPointerException("qualifier is null");
1196 }
1197 if (npe != null) {
1198 throw new IOException(
1199 "Invalid arguments to incrementColumnValue", npe);
1200 }
1201
1202 NonceGenerator ng = this.connection.getNonceGenerator();
1203 final long nonceGroup = ng.getNonceGroup(), nonce = ng.newNonce();
1204 RegionServerCallable<Long> callable =
1205 new RegionServerCallable<Long>(connection, getName(), row) {
1206 public Long call() throws IOException {
1207 try {
1208 MutateRequest request = RequestConverter.buildIncrementRequest(
1209 getLocation().getRegionInfo().getRegionName(), row, family,
1210 qualifier, amount, durability, nonceGroup, nonce);
1211 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1212 rpcController.setPriority(getTableName());
1213 MutateResponse response = getStub().mutate(rpcController, request);
1214 Result result =
1215 ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1216 return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
1217 } catch (ServiceException se) {
1218 throw ProtobufUtil.getRemoteException(se);
1219 }
1220 }
1221 };
1222 return rpcCallerFactory.<Long> newCaller().callWithRetries(callable, this.operationTimeout);
1223 }
1224
1225
1226
1227
1228 @Override
1229 public boolean checkAndPut(final byte [] row,
1230 final byte [] family, final byte [] qualifier, final byte [] value,
1231 final Put put)
1232 throws IOException {
1233 RegionServerCallable<Boolean> callable =
1234 new RegionServerCallable<Boolean>(connection, getName(), row) {
1235 public Boolean call() throws IOException {
1236 try {
1237 MutateRequest request = RequestConverter.buildMutateRequest(
1238 getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
1239 new BinaryComparator(value), CompareType.EQUAL, put);
1240 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1241 rpcController.setPriority(getTableName());
1242 MutateResponse response = getStub().mutate(rpcController, request);
1243 return Boolean.valueOf(response.getProcessed());
1244 } catch (ServiceException se) {
1245 throw ProtobufUtil.getRemoteException(se);
1246 }
1247 }
1248 };
1249 return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
1250 }
1251
1252
1253
1254
1255
1256 @Override
1257 public boolean checkAndDelete(final byte [] row,
1258 final byte [] family, final byte [] qualifier, final byte [] value,
1259 final Delete delete)
1260 throws IOException {
1261 RegionServerCallable<Boolean> callable =
1262 new RegionServerCallable<Boolean>(connection, getName(), row) {
1263 public Boolean call() throws IOException {
1264 try {
1265 MutateRequest request = RequestConverter.buildMutateRequest(
1266 getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
1267 new BinaryComparator(value), CompareType.EQUAL, delete);
1268 PayloadCarryingRpcController rpcController = rpcControllerFactory.newController();
1269 rpcController.setPriority(getTableName());
1270 MutateResponse response = getStub().mutate(rpcController, request);
1271 return Boolean.valueOf(response.getProcessed());
1272 } catch (ServiceException se) {
1273 throw ProtobufUtil.getRemoteException(se);
1274 }
1275 }
1276 };
1277 return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
1278 }
1279
1280
1281
1282
1283 @Override
1284 public boolean checkAndMutate(final byte [] row, final byte [] family, final byte [] qualifier,
1285 final CompareOp compareOp, final byte [] value, final RowMutations rm)
1286 throws IOException {
1287 final RetryingTimeTracker tracker = new RetryingTimeTracker();
1288 PayloadCarryingServerCallable<MultiResponse> callable =
1289 new PayloadCarryingServerCallable<MultiResponse>(connection, getName(), rm.getRow(),
1290 rpcControllerFactory) {
1291 @Override
1292 public MultiResponse call() throws IOException {
1293 tracker.start();
1294 controller.setPriority(tableName);
1295 int remainingTime = tracker.getRemainingTime(operationTimeout);
1296 if (remainingTime == 0) {
1297 throw new DoNotRetryIOException("Timeout for mutate row");
1298 }
1299 RpcClient.setRpcTimeout(remainingTime);
1300 try {
1301 RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
1302 getLocation().getRegionInfo().getRegionName(), rm);
1303 regionMutationBuilder.setAtomic(true);
1304 MultiRequest request =
1305 MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
1306 ClientProtos.MultiResponse response = getStub().multi(controller, request);
1307 response.getRegionActionResult(0).getResultOrException(0).getResult();
1308 ClientProtos.RegionActionResult res = response.getRegionActionResultList().get(0);
1309 if (res.hasException()) {
1310 Throwable ex = ProtobufUtil.toException(res.getException());
1311 if (ex instanceof IOException) {
1312 throw (IOException) ex;
1313 }
1314 throw new IOException("Failed to mutate row: " +
1315 Bytes.toStringBinary(rm.getRow()), ex);
1316 }
1317 return ResponseConverter.getResults(request, response, controller.cellScanner());
1318 } catch (ServiceException se) {
1319 throw ProtobufUtil.getRemoteException(se);
1320 }
1321 }
1322 };
1323
1324
1325
1326
1327 final boolean[] processed = new boolean[1];
1328 ap.submitAll(rm.getMutations(), new Batch.Callback<Object>() {
1329 @Override
1330 public void update(byte[] region, byte[] row, Object result) {
1331 processed[0] = ((Result)result).getExists();
1332 }
1333 }, callable);
1334 ap.waitUntilDone();
1335 try {
1336 if (ap.hasError()) {
1337 throw ap.getErrors();
1338 }
1339 } finally {
1340 ap.clearErrors();
1341 }
1342 return processed[0];
1343 }
1344
1345
1346
1347
1348 @Override
1349 public boolean exists(final Get get) throws IOException {
1350 Result r = get(get, true);
1351 assert r.getExists() != null;
1352 return r.getExists();
1353 }
1354
1355
1356
1357
1358 @Override
1359 public Boolean[] exists(final List<Get> gets) throws IOException {
1360 if (gets.isEmpty()) return new Boolean[]{};
1361 if (gets.size() == 1) return new Boolean[]{exists(gets.get(0))};
1362
1363 ArrayList<Get> exists = new ArrayList<Get>(gets.size());
1364 for (Get g: gets){
1365 Get ge = new Get(g);
1366 ge.setCheckExistenceOnly(true);
1367 exists.add(ge);
1368 }
1369
1370 Object[] r1;
1371 try {
1372 r1 = batch(exists);
1373 } catch (InterruptedException e) {
1374 throw (InterruptedIOException)new InterruptedIOException().initCause(e);
1375 }
1376
1377
1378 Boolean[] results = new Boolean[r1.length];
1379 int i = 0;
1380 for (Object o : r1) {
1381
1382 results[i++] = ((Result)o).getExists();
1383 }
1384
1385 return results;
1386 }
1387
1388
1389
1390
1391 @Override
1392 public void flushCommits() throws InterruptedIOException, RetriesExhaustedWithDetailsException {
1393
1394
1395 backgroundFlushCommits(true);
1396 }
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409 public <R> void processBatchCallback(
1410 final List<? extends Row> list, final Object[] results, final Batch.Callback<R> callback)
1411 throws IOException, InterruptedException {
1412 this.batchCallback(list, results, callback);
1413 }
1414
1415
1416
1417
1418
1419
1420 public void processBatch(final List<? extends Row> list, final Object[] results)
1421 throws IOException, InterruptedException {
1422
1423 this.processBatchCallback(list, results, null);
1424 }
1425
1426
1427 @Override
1428 public void close() throws IOException {
1429 if (this.closed) {
1430 return;
1431 }
1432 flushCommits();
1433 if (cleanupPoolOnClose) {
1434 this.pool.shutdown();
1435 }
1436 if (cleanupConnectionOnClose) {
1437 if (this.connection != null) {
1438 this.connection.close();
1439 }
1440 }
1441 this.closed = true;
1442 }
1443
1444
1445 public void validatePut(final Put put) throws IllegalArgumentException {
1446 validatePut(put, tableConfiguration.getMaxKeyValueSize());
1447 }
1448
1449
1450 public static void validatePut(Put put, int maxKeyValueSize) throws IllegalArgumentException {
1451 if (put.isEmpty()) {
1452 throw new IllegalArgumentException("No columns to insert");
1453 }
1454 if (maxKeyValueSize > 0) {
1455 for (List<Cell> list : put.getFamilyCellMap().values()) {
1456 for (Cell cell : list) {
1457
1458 KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
1459 if (kv.getLength() > maxKeyValueSize) {
1460 throw new IllegalArgumentException("KeyValue size too large");
1461 }
1462 }
1463 }
1464 }
1465 }
1466
1467
1468
1469
1470 @Override
1471 public boolean isAutoFlush() {
1472 return autoFlush;
1473 }
1474
1475
1476
1477
1478 @Deprecated
1479 @Override
1480 public void setAutoFlush(boolean autoFlush) {
1481 setAutoFlush(autoFlush, autoFlush);
1482 }
1483
1484
1485
1486
1487 @Override
1488 public void setAutoFlushTo(boolean autoFlush) {
1489 setAutoFlush(autoFlush, clearBufferOnFail);
1490 }
1491
1492
1493
1494
1495 @Override
1496 public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
1497 this.autoFlush = autoFlush;
1498 this.clearBufferOnFail = autoFlush || clearBufferOnFail;
1499 }
1500
1501
1502
1503
1504
1505
1506
1507
1508 @Override
1509 public long getWriteBufferSize() {
1510 return writeBufferSize;
1511 }
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521 public void setWriteBufferSize(long writeBufferSize) throws IOException {
1522 this.writeBufferSize = writeBufferSize;
1523 if(currentWriteBufferSize > writeBufferSize) {
1524 flushCommits();
1525 }
1526 }
1527
1528
1529
1530
1531
1532 ExecutorService getPool() {
1533 return this.pool;
1534 }
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545 public static void setRegionCachePrefetch(final byte[] tableName,
1546 final boolean enable) throws IOException {
1547 setRegionCachePrefetch(TableName.valueOf(tableName), enable);
1548 }
1549
1550 public static void setRegionCachePrefetch(
1551 final TableName tableName,
1552 final boolean enable) throws IOException {
1553 HConnectionManager.execute(new HConnectable<Void>(HBaseConfiguration.create()) {
1554 @Override
1555 public Void connect(HConnection connection) throws IOException {
1556 connection.setRegionCachePrefetch(tableName, enable);
1557 return null;
1558 }
1559 });
1560 }
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572 public static void setRegionCachePrefetch(final Configuration conf,
1573 final byte[] tableName, final boolean enable) throws IOException {
1574 setRegionCachePrefetch(conf, TableName.valueOf(tableName), enable);
1575 }
1576
1577 public static void setRegionCachePrefetch(final Configuration conf,
1578 final TableName tableName,
1579 final boolean enable) throws IOException {
1580 HConnectionManager.execute(new HConnectable<Void>(conf) {
1581 @Override
1582 public Void connect(HConnection connection) throws IOException {
1583 connection.setRegionCachePrefetch(tableName, enable);
1584 return null;
1585 }
1586 });
1587 }
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597 public static boolean getRegionCachePrefetch(final Configuration conf,
1598 final byte[] tableName) throws IOException {
1599 return getRegionCachePrefetch(conf, TableName.valueOf(tableName));
1600 }
1601
1602 public static boolean getRegionCachePrefetch(final Configuration conf,
1603 final TableName tableName) throws IOException {
1604 return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
1605 @Override
1606 public Boolean connect(HConnection connection) throws IOException {
1607 return connection.getRegionCachePrefetch(tableName);
1608 }
1609 });
1610 }
1611
1612
1613
1614
1615
1616
1617
1618
1619 public static boolean getRegionCachePrefetch(final byte[] tableName) throws IOException {
1620 return getRegionCachePrefetch(TableName.valueOf(tableName));
1621 }
1622
1623 public static boolean getRegionCachePrefetch(
1624 final TableName tableName) throws IOException {
1625 return HConnectionManager.execute(new HConnectable<Boolean>(
1626 HBaseConfiguration.create()) {
1627 @Override
1628 public Boolean connect(HConnection connection) throws IOException {
1629 return connection.getRegionCachePrefetch(tableName);
1630 }
1631 });
1632 }
1633
1634
1635
1636
1637
1638 public void clearRegionCache() {
1639 this.connection.clearRegionCache();
1640 }
1641
1642
1643
1644
1645 public CoprocessorRpcChannel coprocessorService(byte[] row) {
1646 return new RegionCoprocessorRpcChannel(connection, tableName, row, rpcCallerFactory,
1647 rpcControllerFactory);
1648 }
1649
1650
1651
1652
1653 @Override
1654 public <T extends Service, R> Map<byte[],R> coprocessorService(final Class<T> service,
1655 byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable)
1656 throws ServiceException, Throwable {
1657 final Map<byte[],R> results = Collections.synchronizedMap(
1658 new TreeMap<byte[], R>(Bytes.BYTES_COMPARATOR));
1659 coprocessorService(service, startKey, endKey, callable, new Batch.Callback<R>() {
1660 public void update(byte[] region, byte[] row, R value) {
1661 if (region != null) {
1662 results.put(region, value);
1663 }
1664 }
1665 });
1666 return results;
1667 }
1668
1669
1670
1671
1672 @Override
1673 public <T extends Service, R> void coprocessorService(final Class<T> service,
1674 byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable,
1675 final Batch.Callback<R> callback) throws ServiceException, Throwable {
1676
1677
1678 List<byte[]> keys = getStartKeysInRange(startKey, endKey);
1679
1680 Map<byte[],Future<R>> futures =
1681 new TreeMap<byte[],Future<R>>(Bytes.BYTES_COMPARATOR);
1682 for (final byte[] r : keys) {
1683 final RegionCoprocessorRpcChannel channel =
1684 new RegionCoprocessorRpcChannel(connection, tableName, r, rpcCallerFactory,
1685 rpcControllerFactory);
1686 Future<R> future = pool.submit(
1687 new Callable<R>() {
1688 public R call() throws Exception {
1689 T instance = ProtobufUtil.newServiceStub(service, channel);
1690 R result = callable.call(instance);
1691 byte[] region = channel.getLastRegion();
1692 if (callback != null) {
1693 callback.update(region, r, result);
1694 }
1695 return result;
1696 }
1697 });
1698 futures.put(r, future);
1699 }
1700 for (Map.Entry<byte[],Future<R>> e : futures.entrySet()) {
1701 try {
1702 e.getValue().get();
1703 } catch (ExecutionException ee) {
1704 LOG.warn("Error calling coprocessor service " + service.getName() + " for row "
1705 + Bytes.toStringBinary(e.getKey()), ee);
1706 throw ee.getCause();
1707 } catch (InterruptedException ie) {
1708 throw new InterruptedIOException("Interrupted calling coprocessor service " + service.getName()
1709 + " for row " + Bytes.toStringBinary(e.getKey()))
1710 .initCause(ie);
1711 }
1712 }
1713 }
1714
1715 private List<byte[]> getStartKeysInRange(byte[] start, byte[] end)
1716 throws IOException {
1717 if (start == null) {
1718 start = HConstants.EMPTY_START_ROW;
1719 }
1720 if (end == null) {
1721 end = HConstants.EMPTY_END_ROW;
1722 }
1723 return getKeysAndRegionsInRange(start, end, true).getFirst();
1724 }
1725
1726 public void setOperationTimeout(int operationTimeout) {
1727 this.operationTimeout = operationTimeout;
1728 }
1729
1730 public int getOperationTimeout() {
1731 return operationTimeout;
1732 }
1733
1734 @Override
1735 public String toString() {
1736 return tableName + ";" + connection;
1737 }
1738
1739
1740
1741
1742
1743
1744 public static void main(String[] args) throws IOException {
1745 HTable t = new HTable(HBaseConfiguration.create(), args[0]);
1746 try {
1747 System.out.println(t.get(new Get(Bytes.toBytes(args[1]))));
1748 } finally {
1749 t.close();
1750 }
1751 }
1752
1753
1754
1755
1756 @Override
1757 public <R extends Message> Map<byte[], R> batchCoprocessorService(
1758 Descriptors.MethodDescriptor methodDescriptor, Message request,
1759 byte[] startKey, byte[] endKey, R responsePrototype) throws ServiceException, Throwable {
1760 final Map<byte[], R> results = Collections.synchronizedMap(new TreeMap<byte[], R>(
1761 Bytes.BYTES_COMPARATOR));
1762 batchCoprocessorService(methodDescriptor, request, startKey, endKey, responsePrototype,
1763 new Callback<R>() {
1764
1765 @Override
1766 public void update(byte[] region, byte[] row, R result) {
1767 if (region != null) {
1768 results.put(region, result);
1769 }
1770 }
1771 });
1772 return results;
1773 }
1774
1775
1776
1777
1778 @Override
1779 public <R extends Message> void batchCoprocessorService(
1780 final Descriptors.MethodDescriptor methodDescriptor, final Message request,
1781 byte[] startKey, byte[] endKey, final R responsePrototype, final Callback<R> callback)
1782 throws ServiceException, Throwable {
1783
1784 if (startKey == null) {
1785 startKey = HConstants.EMPTY_START_ROW;
1786 }
1787 if (endKey == null) {
1788 endKey = HConstants.EMPTY_END_ROW;
1789 }
1790
1791 Pair<List<byte[]>, List<HRegionLocation>> keysAndRegions =
1792 getKeysAndRegionsInRange(startKey, endKey, true);
1793 List<byte[]> keys = keysAndRegions.getFirst();
1794 List<HRegionLocation> regions = keysAndRegions.getSecond();
1795
1796
1797 if (keys.isEmpty()) {
1798 LOG.info("No regions were selected by key range start=" + Bytes.toStringBinary(startKey) +
1799 ", end=" + Bytes.toStringBinary(endKey));
1800 return;
1801 }
1802
1803 List<RegionCoprocessorServiceExec> execs = new ArrayList<RegionCoprocessorServiceExec>();
1804 final Map<byte[], RegionCoprocessorServiceExec> execsByRow =
1805 new TreeMap<byte[], RegionCoprocessorServiceExec>(Bytes.BYTES_COMPARATOR);
1806 for (int i = 0; i < keys.size(); i++) {
1807 final byte[] rowKey = keys.get(i);
1808 final byte[] region = regions.get(i).getRegionInfo().getRegionName();
1809 RegionCoprocessorServiceExec exec =
1810 new RegionCoprocessorServiceExec(region, rowKey, methodDescriptor, request);
1811 execs.add(exec);
1812 execsByRow.put(rowKey, exec);
1813 }
1814
1815
1816
1817 final List<Throwable> callbackErrorExceptions = new ArrayList<Throwable>();
1818 final List<Row> callbackErrorActions = new ArrayList<Row>();
1819 final List<String> callbackErrorServers = new ArrayList<String>();
1820
1821 AsyncProcess<ClientProtos.CoprocessorServiceResult> asyncProcess =
1822 new AsyncProcess<ClientProtos.CoprocessorServiceResult>(connection, tableName, pool,
1823 new AsyncProcess.AsyncProcessCallback<ClientProtos.CoprocessorServiceResult>() {
1824 @SuppressWarnings("unchecked")
1825 @Override
1826 public void success(int originalIndex, byte[] region, Row row,
1827 ClientProtos.CoprocessorServiceResult serviceResult) {
1828 if (LOG.isTraceEnabled()) {
1829 LOG.trace("Received result for endpoint " + methodDescriptor.getFullName() +
1830 " call #" + originalIndex + ": region=" + Bytes.toStringBinary(region) +
1831 ", row=" + Bytes.toStringBinary(row.getRow()) +
1832 ", value=" + serviceResult.getValue().getValue());
1833 }
1834 try {
1835 Message.Builder builder = responsePrototype.newBuilderForType();
1836 ProtobufUtil.mergeFrom(builder, serviceResult.getValue().getValue());
1837 callback.update(region, row.getRow(), (R) builder.build());
1838 } catch (IOException e) {
1839 LOG.error("Unexpected response type from endpoint " + methodDescriptor.getFullName(),
1840 e);
1841 callbackErrorExceptions.add(e);
1842 callbackErrorActions.add(row);
1843 callbackErrorServers.add("null");
1844 }
1845 }
1846
1847 @Override
1848 public boolean failure(int originalIndex, byte[] region, Row row, Throwable t) {
1849 RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
1850 LOG.error("Failed calling endpoint " + methodDescriptor.getFullName() + ": region="
1851 + Bytes.toStringBinary(exec.getRegion()), t);
1852 return true;
1853 }
1854
1855 @Override
1856 public boolean retriableFailure(int originalIndex, Row row, byte[] region,
1857 Throwable exception) {
1858 RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
1859 LOG.error("Failed calling endpoint " + methodDescriptor.getFullName() + ": region="
1860 + Bytes.toStringBinary(exec.getRegion()), exception);
1861 return !(exception instanceof DoNotRetryIOException);
1862 }
1863 },
1864 configuration, rpcCallerFactory, rpcControllerFactory);
1865
1866 asyncProcess.submitAll(execs);
1867 asyncProcess.waitUntilDone();
1868
1869 if (asyncProcess.hasError()) {
1870 throw asyncProcess.getErrors();
1871 } else if (!callbackErrorExceptions.isEmpty()) {
1872 throw new RetriesExhaustedWithDetailsException(callbackErrorExceptions, callbackErrorActions,
1873 callbackErrorServers);
1874 }
1875 }
1876 }