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