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