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()) {
717 return new ClientSmallScanner(getConfiguration(), scan, getName(),
718 this.connection);
719 }
720 return new ClientScanner(getConfiguration(), scan,
721 getName(), this.connection);
722 }
723
724
725
726
727 @Override
728 public ResultScanner getScanner(byte [] family) throws IOException {
729 Scan scan = new Scan();
730 scan.addFamily(family);
731 return getScanner(scan);
732 }
733
734
735
736
737 @Override
738 public ResultScanner getScanner(byte [] family, byte [] qualifier)
739 throws IOException {
740 Scan scan = new Scan();
741 scan.addColumn(family, qualifier);
742 return getScanner(scan);
743 }
744
745
746
747
748 @Override
749 public Result get(final Get get) throws IOException {
750 RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
751 getName(), get.getRow()) {
752 public Result call() throws IOException {
753 return ProtobufUtil.get(getStub(), getLocation().getRegionInfo().getRegionName(), get);
754 }
755 };
756 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
757 }
758
759
760
761
762 @Override
763 public Result[] get(List<Get> gets) throws IOException {
764 if (gets.size() == 1) {
765 return new Result[]{get(gets.get(0))};
766 }
767 try {
768 Object [] r1 = batch((List)gets);
769
770
771 Result [] results = new Result[r1.length];
772 int i=0;
773 for (Object o : r1) {
774
775 results[i++] = (Result) o;
776 }
777
778 return results;
779 } catch (InterruptedException e) {
780 throw new IOException(e);
781 }
782 }
783
784 @Override
785 public void batch(final List<?extends Row> actions, final Object[] results)
786 throws InterruptedException, IOException {
787 batchCallback(actions, results, null);
788 }
789
790 @Override
791 public Object[] batch(final List<? extends Row> actions)
792 throws InterruptedException, IOException {
793 return batchCallback(actions, null);
794 }
795
796 @Override
797 public <R> void batchCallback(
798 final List<? extends Row> actions, final Object[] results, final Batch.Callback<R> callback)
799 throws IOException, InterruptedException {
800 connection.processBatchCallback(actions, tableName, pool, results, callback);
801 }
802
803 @Override
804 public <R> Object[] batchCallback(
805 final List<? extends Row> actions, final Batch.Callback<R> callback) throws IOException,
806 InterruptedException {
807 Object[] results = new Object[actions.size()];
808 batchCallback(actions, results, callback);
809 return results;
810 }
811
812
813
814
815 @Override
816 public void delete(final Delete delete)
817 throws IOException {
818 RegionServerCallable<Boolean> callable = new RegionServerCallable<Boolean>(connection,
819 tableName, delete.getRow()) {
820 public Boolean call() throws IOException {
821 try {
822 MutateRequest request = RequestConverter.buildMutateRequest(
823 getLocation().getRegionInfo().getRegionName(), delete);
824 MutateResponse response = getStub().mutate(null, request);
825 return Boolean.valueOf(response.getProcessed());
826 } catch (ServiceException se) {
827 throw ProtobufUtil.getRemoteException(se);
828 }
829 }
830 };
831 rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
832 }
833
834
835
836
837 @Override
838 public void delete(final List<Delete> deletes)
839 throws IOException {
840 Object[] results = new Object[deletes.size()];
841 try {
842 batch(deletes, results);
843 } catch (InterruptedException e) {
844 throw new IOException(e);
845 } finally {
846
847
848
849 for (int i = results.length - 1; i>=0; i--) {
850
851 if (results[i] instanceof Result) {
852 deletes.remove(i);
853 }
854 }
855 }
856 }
857
858
859
860
861 @Override
862 public void put(final Put put)
863 throws InterruptedIOException, RetriesExhaustedWithDetailsException {
864 doPut(put);
865 if (autoFlush) {
866 flushCommits();
867 }
868 }
869
870
871
872
873 @Override
874 public void put(final List<Put> puts)
875 throws InterruptedIOException, RetriesExhaustedWithDetailsException {
876 for (Put put : puts) {
877 doPut(put);
878 }
879 if (autoFlush) {
880 flushCommits();
881 }
882 }
883
884
885
886
887
888
889
890
891 private void doPut(Put put) throws InterruptedIOException, RetriesExhaustedWithDetailsException {
892 if (ap.hasError()){
893 writeAsyncBuffer.add(put);
894 backgroundFlushCommits(true);
895 }
896
897 validatePut(put);
898
899 currentWriteBufferSize += put.heapSize();
900 writeAsyncBuffer.add(put);
901
902 while (currentWriteBufferSize > writeBufferSize) {
903 backgroundFlushCommits(false);
904 }
905 }
906
907
908
909
910
911
912
913
914
915 private void backgroundFlushCommits(boolean synchronous) throws
916 InterruptedIOException, RetriesExhaustedWithDetailsException {
917
918 try {
919 do {
920 ap.submit(writeAsyncBuffer, true);
921 } while (synchronous && !writeAsyncBuffer.isEmpty());
922
923 if (synchronous) {
924 ap.waitUntilDone();
925 }
926
927 if (ap.hasError()) {
928 LOG.debug(tableName + ": One or more of the operations have failed -" +
929 " waiting for all operation in progress to finish (successfully or not)");
930 while (!writeAsyncBuffer.isEmpty()) {
931 ap.submit(writeAsyncBuffer, true);
932 }
933 ap.waitUntilDone();
934
935 if (!clearBufferOnFail) {
936
937
938 writeAsyncBuffer.addAll(ap.getFailedOperations());
939 }
940 RetriesExhaustedWithDetailsException e = ap.getErrors();
941 ap.clearErrors();
942 throw e;
943 }
944 } finally {
945 currentWriteBufferSize = 0;
946 for (Row mut : writeAsyncBuffer) {
947 if (mut instanceof Mutation) {
948 currentWriteBufferSize += ((Mutation) mut).heapSize();
949 }
950 }
951 }
952 }
953
954
955
956
957 @Override
958 public void mutateRow(final RowMutations rm) throws IOException {
959 RegionServerCallable<Void> callable =
960 new RegionServerCallable<Void>(connection, getName(), rm.getRow()) {
961 public Void call() throws IOException {
962 try {
963 RegionAction.Builder regionMutationBuilder = RequestConverter.buildRegionAction(
964 getLocation().getRegionInfo().getRegionName(), rm);
965 regionMutationBuilder.setAtomic(true);
966 MultiRequest request =
967 MultiRequest.newBuilder().addRegionAction(regionMutationBuilder.build()).build();
968 PayloadCarryingRpcController pcrc = new PayloadCarryingRpcController();
969 pcrc.setPriority(tableName);
970 getStub().multi(null, request);
971 } catch (ServiceException se) {
972 throw ProtobufUtil.getRemoteException(se);
973 }
974 return null;
975 }
976 };
977 rpcCallerFactory.<Void> newCaller().callWithRetries(callable, this.operationTimeout);
978 }
979
980
981
982
983 @Override
984 public Result append(final Append append) throws IOException {
985 if (append.numFamilies() == 0) {
986 throw new IOException(
987 "Invalid arguments to append, no columns specified");
988 }
989 RegionServerCallable<Result> callable =
990 new RegionServerCallable<Result>(this.connection, getName(), append.getRow()) {
991 public Result call() throws IOException {
992 try {
993 MutateRequest request = RequestConverter.buildMutateRequest(
994 getLocation().getRegionInfo().getRegionName(), append);
995 PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
996 rpcController.setPriority(getTableName());
997 MutateResponse response = getStub().mutate(rpcController, request);
998 if (!response.hasResult()) return null;
999 return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1000 } catch (ServiceException se) {
1001 throw ProtobufUtil.getRemoteException(se);
1002 }
1003 }
1004 };
1005 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
1006 }
1007
1008
1009
1010
1011 @Override
1012 public Result increment(final Increment increment) throws IOException {
1013 if (!increment.hasFamilies()) {
1014 throw new IOException(
1015 "Invalid arguments to increment, no columns specified");
1016 }
1017 RegionServerCallable<Result> callable = new RegionServerCallable<Result>(this.connection,
1018 getName(), increment.getRow()) {
1019 public Result call() throws IOException {
1020 try {
1021 MutateRequest request = RequestConverter.buildMutateRequest(
1022 getLocation().getRegionInfo().getRegionName(), increment);
1023 PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
1024 rpcController.setPriority(getTableName());
1025 MutateResponse response = getStub().mutate(rpcController, request);
1026 return ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1027 } catch (ServiceException se) {
1028 throw ProtobufUtil.getRemoteException(se);
1029 }
1030 }
1031 };
1032 return rpcCallerFactory.<Result> newCaller().callWithRetries(callable, this.operationTimeout);
1033 }
1034
1035
1036
1037
1038 @Override
1039 public long incrementColumnValue(final byte [] row, final byte [] family,
1040 final byte [] qualifier, final long amount)
1041 throws IOException {
1042 return incrementColumnValue(row, family, qualifier, amount, Durability.SYNC_WAL);
1043 }
1044
1045
1046
1047
1048 @Deprecated
1049 @Override
1050 public long incrementColumnValue(final byte [] row, final byte [] family,
1051 final byte [] qualifier, final long amount, final boolean writeToWAL)
1052 throws IOException {
1053 return incrementColumnValue(row, family, qualifier, amount,
1054 writeToWAL? Durability.SKIP_WAL: Durability.USE_DEFAULT);
1055 }
1056
1057
1058
1059
1060 @Override
1061 public long incrementColumnValue(final byte [] row, final byte [] family,
1062 final byte [] qualifier, final long amount, final Durability durability)
1063 throws IOException {
1064 NullPointerException npe = null;
1065 if (row == null) {
1066 npe = new NullPointerException("row is null");
1067 } else if (family == null) {
1068 npe = new NullPointerException("family is null");
1069 } else if (qualifier == null) {
1070 npe = new NullPointerException("qualifier is null");
1071 }
1072 if (npe != null) {
1073 throw new IOException(
1074 "Invalid arguments to incrementColumnValue", npe);
1075 }
1076
1077 RegionServerCallable<Long> callable =
1078 new RegionServerCallable<Long>(connection, getName(), row) {
1079 public Long call() throws IOException {
1080 try {
1081 MutateRequest request = RequestConverter.buildMutateRequest(
1082 getLocation().getRegionInfo().getRegionName(), row, family,
1083 qualifier, amount, durability);
1084 PayloadCarryingRpcController rpcController = new PayloadCarryingRpcController();
1085 rpcController.setPriority(getTableName());
1086 MutateResponse response = getStub().mutate(rpcController, request);
1087 Result result =
1088 ProtobufUtil.toResult(response.getResult(), rpcController.cellScanner());
1089 return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier)));
1090 } catch (ServiceException se) {
1091 throw ProtobufUtil.getRemoteException(se);
1092 }
1093 }
1094 };
1095 return rpcCallerFactory.<Long> newCaller().callWithRetries(callable, this.operationTimeout);
1096 }
1097
1098
1099
1100
1101 @Override
1102 public boolean checkAndPut(final byte [] row,
1103 final byte [] family, final byte [] qualifier, final byte [] value,
1104 final Put put)
1105 throws IOException {
1106 RegionServerCallable<Boolean> callable =
1107 new RegionServerCallable<Boolean>(connection, getName(), row) {
1108 public Boolean call() throws IOException {
1109 try {
1110 MutateRequest request = RequestConverter.buildMutateRequest(
1111 getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
1112 new BinaryComparator(value), CompareType.EQUAL, put);
1113 MutateResponse response = getStub().mutate(null, request);
1114 return Boolean.valueOf(response.getProcessed());
1115 } catch (ServiceException se) {
1116 throw ProtobufUtil.getRemoteException(se);
1117 }
1118 }
1119 };
1120 return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
1121 }
1122
1123
1124
1125
1126
1127 @Override
1128 public boolean checkAndDelete(final byte [] row,
1129 final byte [] family, final byte [] qualifier, final byte [] value,
1130 final Delete delete)
1131 throws IOException {
1132 RegionServerCallable<Boolean> callable =
1133 new RegionServerCallable<Boolean>(connection, getName(), row) {
1134 public Boolean call() throws IOException {
1135 try {
1136 MutateRequest request = RequestConverter.buildMutateRequest(
1137 getLocation().getRegionInfo().getRegionName(), row, family, qualifier,
1138 new BinaryComparator(value), CompareType.EQUAL, delete);
1139 MutateResponse response = getStub().mutate(null, request);
1140 return Boolean.valueOf(response.getProcessed());
1141 } catch (ServiceException se) {
1142 throw ProtobufUtil.getRemoteException(se);
1143 }
1144 }
1145 };
1146 return rpcCallerFactory.<Boolean> newCaller().callWithRetries(callable, this.operationTimeout);
1147 }
1148
1149
1150
1151
1152 @Override
1153 public boolean exists(final Get get) throws IOException {
1154 get.setCheckExistenceOnly(true);
1155 Result r = get(get);
1156 assert r.getExists() != null;
1157 return r.getExists();
1158 }
1159
1160
1161
1162
1163 @Override
1164 public Boolean[] exists(final List<Get> gets) throws IOException {
1165 if (gets.isEmpty()) return new Boolean[]{};
1166 if (gets.size() == 1) return new Boolean[]{exists(gets.get(0))};
1167
1168 for (Get g: gets){
1169 g.setCheckExistenceOnly(true);
1170 }
1171
1172 Object[] r1;
1173 try {
1174 r1 = batch(gets);
1175 } catch (InterruptedException e) {
1176 throw new IOException(e);
1177 }
1178
1179
1180 Boolean[] results = new Boolean[r1.length];
1181 int i = 0;
1182 for (Object o : r1) {
1183
1184 results[i++] = ((Result)o).getExists();
1185 }
1186
1187 return results;
1188 }
1189
1190
1191
1192
1193 @Override
1194 public void flushCommits() throws InterruptedIOException, RetriesExhaustedWithDetailsException {
1195
1196
1197 backgroundFlushCommits(true);
1198 }
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211 public <R> void processBatchCallback(
1212 final List<? extends Row> list, final Object[] results, final Batch.Callback<R> callback)
1213 throws IOException, InterruptedException {
1214 this.batchCallback(list, results, callback);
1215 }
1216
1217
1218
1219
1220
1221
1222 public void processBatch(final List<? extends Row> list, final Object[] results)
1223 throws IOException, InterruptedException {
1224
1225 this.processBatchCallback(list, results, null);
1226 }
1227
1228
1229 @Override
1230 public void close() throws IOException {
1231 if (this.closed) {
1232 return;
1233 }
1234 flushCommits();
1235 if (cleanupPoolOnClose && this.pool != null) {
1236 this.pool.shutdown();
1237 }
1238 if (cleanupConnectionOnClose) {
1239 if (this.connection != null) {
1240 this.connection.close();
1241 }
1242 }
1243 this.closed = true;
1244 }
1245
1246
1247 public void validatePut(final Put put) throws IllegalArgumentException{
1248 if (put.isEmpty()) {
1249 throw new IllegalArgumentException("No columns to insert");
1250 }
1251 if (maxKeyValueSize > 0) {
1252 for (List<Cell> list : put.getFamilyCellMap().values()) {
1253 for (Cell cell : list) {
1254
1255 KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
1256 if (kv.getLength() > maxKeyValueSize) {
1257 throw new IllegalArgumentException("KeyValue size too large");
1258 }
1259 }
1260 }
1261 }
1262 }
1263
1264
1265
1266
1267 @Override
1268 public boolean isAutoFlush() {
1269 return autoFlush;
1270 }
1271
1272
1273
1274
1275 @Deprecated
1276 @Override
1277 public void setAutoFlush(boolean autoFlush) {
1278 setAutoFlush(autoFlush, autoFlush);
1279 }
1280
1281
1282
1283
1284 @Override
1285 public void setAutoFlushTo(boolean autoFlush) {
1286 setAutoFlush(autoFlush, clearBufferOnFail);
1287 }
1288
1289
1290
1291
1292 @Override
1293 public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
1294 this.autoFlush = autoFlush;
1295 this.clearBufferOnFail = autoFlush || clearBufferOnFail;
1296 }
1297
1298
1299
1300
1301
1302
1303
1304
1305 @Override
1306 public long getWriteBufferSize() {
1307 return writeBufferSize;
1308 }
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318 public void setWriteBufferSize(long writeBufferSize) throws IOException {
1319 this.writeBufferSize = writeBufferSize;
1320 if(currentWriteBufferSize > writeBufferSize) {
1321 flushCommits();
1322 }
1323 }
1324
1325
1326
1327
1328
1329 ExecutorService getPool() {
1330 return this.pool;
1331 }
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342 public static void setRegionCachePrefetch(final byte[] tableName,
1343 final boolean enable) throws IOException {
1344 setRegionCachePrefetch(TableName.valueOf(tableName), enable);
1345 }
1346
1347 public static void setRegionCachePrefetch(
1348 final TableName tableName,
1349 final boolean enable) throws IOException {
1350 HConnectionManager.execute(new HConnectable<Void>(HBaseConfiguration
1351 .create()) {
1352 @Override
1353 public Void connect(HConnection connection) throws IOException {
1354 connection.setRegionCachePrefetch(tableName, enable);
1355 return null;
1356 }
1357 });
1358 }
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370 public static void setRegionCachePrefetch(final Configuration conf,
1371 final byte[] tableName, final boolean enable) throws IOException {
1372 setRegionCachePrefetch(conf, TableName.valueOf(tableName), enable);
1373 }
1374
1375 public static void setRegionCachePrefetch(final Configuration conf,
1376 final TableName tableName,
1377 final boolean enable) throws IOException {
1378 HConnectionManager.execute(new HConnectable<Void>(conf) {
1379 @Override
1380 public Void connect(HConnection connection) throws IOException {
1381 connection.setRegionCachePrefetch(tableName, enable);
1382 return null;
1383 }
1384 });
1385 }
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395 public static boolean getRegionCachePrefetch(final Configuration conf,
1396 final byte[] tableName) throws IOException {
1397 return getRegionCachePrefetch(conf, TableName.valueOf(tableName));
1398 }
1399
1400 public static boolean getRegionCachePrefetch(final Configuration conf,
1401 final TableName tableName) throws IOException {
1402 return HConnectionManager.execute(new HConnectable<Boolean>(conf) {
1403 @Override
1404 public Boolean connect(HConnection connection) throws IOException {
1405 return connection.getRegionCachePrefetch(tableName);
1406 }
1407 });
1408 }
1409
1410
1411
1412
1413
1414
1415
1416
1417 public static boolean getRegionCachePrefetch(final byte[] tableName) throws IOException {
1418 return getRegionCachePrefetch(TableName.valueOf(tableName));
1419 }
1420
1421 public static boolean getRegionCachePrefetch(
1422 final TableName tableName) throws IOException {
1423 return HConnectionManager.execute(new HConnectable<Boolean>(
1424 HBaseConfiguration.create()) {
1425 @Override
1426 public Boolean connect(HConnection connection) throws IOException {
1427 return connection.getRegionCachePrefetch(tableName);
1428 }
1429 });
1430 }
1431
1432
1433
1434
1435
1436 public void clearRegionCache() {
1437 this.connection.clearRegionCache();
1438 }
1439
1440
1441
1442
1443 public CoprocessorRpcChannel coprocessorService(byte[] row) {
1444 return new RegionCoprocessorRpcChannel(connection, tableName, row);
1445 }
1446
1447
1448
1449
1450 @Override
1451 public <T extends Service, R> Map<byte[],R> coprocessorService(final Class<T> service,
1452 byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable)
1453 throws ServiceException, Throwable {
1454 final Map<byte[],R> results = Collections.synchronizedMap(
1455 new TreeMap<byte[], R>(Bytes.BYTES_COMPARATOR));
1456 coprocessorService(service, startKey, endKey, callable, new Batch.Callback<R>() {
1457 public void update(byte[] region, byte[] row, R value) {
1458 results.put(region, value);
1459 }
1460 });
1461 return results;
1462 }
1463
1464
1465
1466
1467 @Override
1468 public <T extends Service, R> void coprocessorService(final Class<T> service,
1469 byte[] startKey, byte[] endKey, final Batch.Call<T,R> callable,
1470 final Batch.Callback<R> callback) throws ServiceException, Throwable {
1471
1472
1473 List<byte[]> keys = getStartKeysInRange(startKey, endKey);
1474
1475 Map<byte[],Future<R>> futures =
1476 new TreeMap<byte[],Future<R>>(Bytes.BYTES_COMPARATOR);
1477 for (final byte[] r : keys) {
1478 final RegionCoprocessorRpcChannel channel =
1479 new RegionCoprocessorRpcChannel(connection, tableName, r);
1480 Future<R> future = pool.submit(
1481 new Callable<R>() {
1482 public R call() throws Exception {
1483 T instance = ProtobufUtil.newServiceStub(service, channel);
1484 R result = callable.call(instance);
1485 byte[] region = channel.getLastRegion();
1486 if (callback != null) {
1487 callback.update(region, r, result);
1488 }
1489 return result;
1490 }
1491 });
1492 futures.put(r, future);
1493 }
1494 for (Map.Entry<byte[],Future<R>> e : futures.entrySet()) {
1495 try {
1496 e.getValue().get();
1497 } catch (ExecutionException ee) {
1498 LOG.warn("Error calling coprocessor service " + service.getName() + " for row "
1499 + Bytes.toStringBinary(e.getKey()), ee);
1500 throw ee.getCause();
1501 } catch (InterruptedException ie) {
1502 Thread.currentThread().interrupt();
1503 throw new InterruptedIOException("Interrupted calling coprocessor service " + service.getName()
1504 + " for row " + Bytes.toStringBinary(e.getKey()))
1505 .initCause(ie);
1506 }
1507 }
1508 }
1509
1510 private List<byte[]> getStartKeysInRange(byte[] start, byte[] end)
1511 throws IOException {
1512 if (start == null) {
1513 start = HConstants.EMPTY_START_ROW;
1514 }
1515 if (end == null) {
1516 end = HConstants.EMPTY_END_ROW;
1517 }
1518 return getKeysAndRegionsInRange(start, end, true).getFirst();
1519 }
1520
1521 public void setOperationTimeout(int operationTimeout) {
1522 this.operationTimeout = operationTimeout;
1523 }
1524
1525 public int getOperationTimeout() {
1526 return operationTimeout;
1527 }
1528
1529 @Override
1530 public String toString() {
1531 return tableName + ";" + connection;
1532 }
1533
1534
1535
1536
1537
1538
1539 public static void main(String[] args) throws IOException {
1540 HTable t = new HTable(HBaseConfiguration.create(), args[0]);
1541 try {
1542 System.out.println(t.get(new Get(Bytes.toBytes(args[1]))));
1543 } finally {
1544 t.close();
1545 }
1546 }
1547 }