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