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