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