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