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