1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.rest.client;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Collection;
25 import java.util.Iterator;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.Set;
29 import java.util.TreeMap;
30
31 import com.google.protobuf.Service;
32 import com.google.protobuf.ServiceException;
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35
36 import org.apache.hadoop.hbase.TableName;
37 import org.apache.hadoop.hbase.client.*;
38 import org.apache.hadoop.hbase.client.coprocessor.Batch;
39 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
40 import org.apache.hadoop.util.StringUtils;
41
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.HBaseConfiguration;
47 import org.apache.hadoop.hbase.HConstants;
48 import org.apache.hadoop.hbase.HTableDescriptor;
49 import org.apache.hadoop.hbase.KeyValue;
50 import org.apache.hadoop.hbase.KeyValueUtil;
51 import org.apache.hadoop.hbase.client.RowMutations;
52 import org.apache.hadoop.hbase.client.Delete;
53 import org.apache.hadoop.hbase.client.Get;
54 import org.apache.hadoop.hbase.client.HTableInterface;
55 import org.apache.hadoop.hbase.client.Increment;
56 import org.apache.hadoop.hbase.client.Put;
57 import org.apache.hadoop.hbase.client.Row;
58 import org.apache.hadoop.hbase.client.Result;
59 import org.apache.hadoop.hbase.client.ResultScanner;
60 import org.apache.hadoop.hbase.client.Scan;
61 import org.apache.hadoop.hbase.io.TimeRange;
62 import org.apache.hadoop.hbase.rest.Constants;
63 import org.apache.hadoop.hbase.rest.model.CellModel;
64 import org.apache.hadoop.hbase.rest.model.CellSetModel;
65 import org.apache.hadoop.hbase.rest.model.RowModel;
66 import org.apache.hadoop.hbase.rest.model.ScannerModel;
67 import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
68 import org.apache.hadoop.hbase.util.Bytes;
69
70
71
72
73 @InterfaceAudience.Public
74 @InterfaceStability.Stable
75 public class RemoteHTable implements HTableInterface {
76
77 private static final Log LOG = LogFactory.getLog(RemoteHTable.class);
78
79 final Client client;
80 final Configuration conf;
81 final byte[] name;
82 final int maxRetries;
83 final long sleepTime;
84
85 @SuppressWarnings("rawtypes")
86 protected String buildRowSpec(final byte[] row, final Map familyMap,
87 final long startTime, final long endTime, final int maxVersions) {
88 StringBuffer sb = new StringBuffer();
89 sb.append('/');
90 sb.append(Bytes.toStringBinary(name));
91 sb.append('/');
92 sb.append(Bytes.toStringBinary(row));
93 Set families = familyMap.entrySet();
94 if (families != null) {
95 Iterator i = familyMap.entrySet().iterator();
96 sb.append('/');
97 while (i.hasNext()) {
98 Map.Entry e = (Map.Entry)i.next();
99 Collection quals = (Collection)e.getValue();
100 if (quals != null && !quals.isEmpty()) {
101 Iterator ii = quals.iterator();
102 while (ii.hasNext()) {
103 sb.append(Bytes.toStringBinary((byte[])e.getKey()));
104 sb.append(':');
105 Object o = ii.next();
106
107 if (o instanceof byte[]) {
108 sb.append(Bytes.toStringBinary((byte[])o));
109 } else if (o instanceof KeyValue) {
110 sb.append(Bytes.toStringBinary(((KeyValue)o).getQualifier()));
111 } else {
112 throw new RuntimeException("object type not handled");
113 }
114 if (ii.hasNext()) {
115 sb.append(',');
116 }
117 }
118 } else {
119 sb.append(Bytes.toStringBinary((byte[])e.getKey()));
120 sb.append(':');
121 }
122 if (i.hasNext()) {
123 sb.append(',');
124 }
125 }
126 }
127 if (startTime != 0 && endTime != Long.MAX_VALUE) {
128 sb.append('/');
129 sb.append(startTime);
130 if (startTime != endTime) {
131 sb.append(',');
132 sb.append(endTime);
133 }
134 } else if (endTime != Long.MAX_VALUE) {
135 sb.append('/');
136 sb.append(endTime);
137 }
138 if (maxVersions > 1) {
139 sb.append("?v=");
140 sb.append(maxVersions);
141 }
142 return sb.toString();
143 }
144
145 protected String buildMultiRowSpec(final byte[][] rows, int maxVersions) {
146 StringBuilder sb = new StringBuilder();
147 sb.append('/');
148 sb.append(Bytes.toStringBinary(name));
149 sb.append("/multiget/");
150 if (rows == null || rows.length == 0) {
151 return sb.toString();
152 }
153 sb.append("?");
154 for(int i=0; i<rows.length; i++) {
155 byte[] rk = rows[i];
156 if (i != 0) {
157 sb.append('&');
158 }
159 sb.append("row=");
160 sb.append(Bytes.toStringBinary(rk));
161 }
162 sb.append("&v=");
163 sb.append(maxVersions);
164
165 return sb.toString();
166 }
167
168 protected Result[] buildResultFromModel(final CellSetModel model) {
169 List<Result> results = new ArrayList<Result>();
170 for (RowModel row: model.getRows()) {
171 List<KeyValue> kvs = new ArrayList<KeyValue>();
172 for (CellModel cell: row.getCells()) {
173 byte[][] split = KeyValue.parseColumn(cell.getColumn());
174 byte[] column = split[0];
175 byte[] qualifier = split.length > 1 ? split[1] : null;
176 kvs.add(new KeyValue(row.getKey(), column, qualifier,
177 cell.getTimestamp(), cell.getValue()));
178 }
179 results.add(new Result(kvs));
180 }
181 return results.toArray(new Result[results.size()]);
182 }
183
184 protected CellSetModel buildModelFromPut(Put put) {
185 RowModel row = new RowModel(put.getRow());
186 long ts = put.getTimeStamp();
187 for (List<Cell> cells: put.getFamilyCellMap().values()) {
188 for (Cell cell: cells) {
189 KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
190 row.addCell(new CellModel(kv.getFamily(), kv.getQualifier(),
191 ts != HConstants.LATEST_TIMESTAMP ? ts : kv.getTimestamp(),
192 kv.getValue()));
193 }
194 }
195 CellSetModel model = new CellSetModel();
196 model.addRow(row);
197 return model;
198 }
199
200
201
202
203
204
205 public RemoteHTable(Client client, String name) {
206 this(client, HBaseConfiguration.create(), Bytes.toBytes(name));
207 }
208
209
210
211
212
213
214
215 public RemoteHTable(Client client, Configuration conf, String name) {
216 this(client, conf, Bytes.toBytes(name));
217 }
218
219
220
221
222
223
224
225 public RemoteHTable(Client client, Configuration conf, byte[] name) {
226 this.client = client;
227 this.conf = conf;
228 this.name = name;
229 this.maxRetries = conf.getInt("hbase.rest.client.max.retries", 10);
230 this.sleepTime = conf.getLong("hbase.rest.client.sleep", 1000);
231 }
232
233 public byte[] getTableName() {
234 return name.clone();
235 }
236
237 @Override
238 public TableName getName() {
239 return TableName.valueOf(name);
240 }
241
242 public Configuration getConfiguration() {
243 return conf;
244 }
245
246 public HTableDescriptor getTableDescriptor() throws IOException {
247 StringBuilder sb = new StringBuilder();
248 sb.append('/');
249 sb.append(Bytes.toStringBinary(name));
250 sb.append('/');
251 sb.append("schema");
252 for (int i = 0; i < maxRetries; i++) {
253 Response response = client.get(sb.toString(), Constants.MIMETYPE_PROTOBUF);
254 int code = response.getCode();
255 switch (code) {
256 case 200:
257 TableSchemaModel schema = new TableSchemaModel();
258 schema.getObjectFromMessage(response.getBody());
259 return schema.getTableDescriptor();
260 case 509:
261 try {
262 Thread.sleep(sleepTime);
263 } catch (InterruptedException e) { }
264 break;
265 default:
266 throw new IOException("schema request returned " + code);
267 }
268 }
269 throw new IOException("schema request timed out");
270 }
271
272 public void close() throws IOException {
273 client.shutdown();
274 }
275
276 public Result get(Get get) throws IOException {
277 TimeRange range = get.getTimeRange();
278 String spec = buildRowSpec(get.getRow(), get.getFamilyMap(),
279 range.getMin(), range.getMax(), get.getMaxVersions());
280 if (get.getFilter() != null) {
281 LOG.warn("filters not supported on gets");
282 }
283 Result[] results = getResults(spec);
284 if (results.length > 0) {
285 if (results.length > 1) {
286 LOG.warn("too many results for get (" + results.length + ")");
287 }
288 return results[0];
289 } else {
290 return new Result();
291 }
292 }
293
294 public Result[] get(List<Get> gets) throws IOException {
295 byte[][] rows = new byte[gets.size()][];
296 int maxVersions = 1;
297 int count = 0;
298
299 for(Get g:gets) {
300
301 if ( count == 0 ) {
302 maxVersions = g.getMaxVersions();
303 } else if (g.getMaxVersions() != maxVersions) {
304 LOG.warn("MaxVersions on Gets do not match, using the first in the list ("+maxVersions+")");
305 }
306
307 if (g.getFilter() != null) {
308 LOG.warn("filters not supported on gets");
309 }
310
311 rows[count] = g.getRow();
312 count ++;
313 }
314
315 String spec = buildMultiRowSpec(rows, maxVersions);
316
317 return getResults(spec);
318 }
319
320 private Result[] getResults(String spec) throws IOException {
321 for (int i = 0; i < maxRetries; i++) {
322 Response response = client.get(spec, Constants.MIMETYPE_PROTOBUF);
323 int code = response.getCode();
324 switch (code) {
325 case 200:
326 CellSetModel model = new CellSetModel();
327 model.getObjectFromMessage(response.getBody());
328 Result[] results = buildResultFromModel(model);
329 if ( results.length > 0) {
330 return results;
331 }
332
333 case 404:
334 return new Result[0];
335
336 case 509:
337 try {
338 Thread.sleep(sleepTime);
339 } catch (InterruptedException e) { }
340 break;
341 default:
342 throw new IOException("get request returned " + code);
343 }
344 }
345 throw new IOException("get request timed out");
346 }
347
348 public boolean exists(Get get) throws IOException {
349 LOG.warn("exists() is really get(), just use get()");
350 Result result = get(get);
351 return (result != null && !(result.isEmpty()));
352 }
353
354
355
356
357
358 public Boolean[] exists(List<Get> gets) throws IOException {
359 LOG.warn("exists(List<Get>) is really list of get() calls, just use get()");
360 Boolean[] results = new Boolean[gets.size()];
361 for (int i = 0; i < results.length; i++) {
362 results[i] = exists(gets.get(i));
363 }
364 return results;
365 }
366
367 public void put(Put put) throws IOException {
368 CellSetModel model = buildModelFromPut(put);
369 StringBuilder sb = new StringBuilder();
370 sb.append('/');
371 sb.append(Bytes.toStringBinary(name));
372 sb.append('/');
373 sb.append(Bytes.toStringBinary(put.getRow()));
374 for (int i = 0; i < maxRetries; i++) {
375 Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
376 model.createProtobufOutput());
377 int code = response.getCode();
378 switch (code) {
379 case 200:
380 return;
381 case 509:
382 try {
383 Thread.sleep(sleepTime);
384 } catch (InterruptedException e) { }
385 break;
386 default:
387 throw new IOException("put request failed with " + code);
388 }
389 }
390 throw new IOException("put request timed out");
391 }
392
393 public void put(List<Put> puts) throws IOException {
394
395
396
397
398 TreeMap<byte[],List<Cell>> map =
399 new TreeMap<byte[],List<Cell>>(Bytes.BYTES_COMPARATOR);
400 for (Put put: puts) {
401 byte[] row = put.getRow();
402 List<Cell> cells = map.get(row);
403 if (cells == null) {
404 cells = new ArrayList<Cell>();
405 map.put(row, cells);
406 }
407 for (List<Cell> l: put.getFamilyCellMap().values()) {
408 cells.addAll(l);
409 }
410 }
411
412
413 CellSetModel model = new CellSetModel();
414 for (Map.Entry<byte[], List<Cell>> e: map.entrySet()) {
415 RowModel row = new RowModel(e.getKey());
416 for (Cell cell: e.getValue()) {
417 KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
418 row.addCell(new CellModel(kv));
419 }
420 model.addRow(row);
421 }
422
423
424 StringBuilder sb = new StringBuilder();
425 sb.append('/');
426 sb.append(Bytes.toStringBinary(name));
427 sb.append("/$multiput");
428 for (int i = 0; i < maxRetries; i++) {
429 Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
430 model.createProtobufOutput());
431 int code = response.getCode();
432 switch (code) {
433 case 200:
434 return;
435 case 509:
436 try {
437 Thread.sleep(sleepTime);
438 } catch (InterruptedException e) { }
439 break;
440 default:
441 throw new IOException("multiput request failed with " + code);
442 }
443 }
444 throw new IOException("multiput request timed out");
445 }
446
447 public void delete(Delete delete) throws IOException {
448 String spec = buildRowSpec(delete.getRow(), delete.getFamilyCellMap(),
449 delete.getTimeStamp(), delete.getTimeStamp(), 1);
450 for (int i = 0; i < maxRetries; i++) {
451 Response response = client.delete(spec);
452 int code = response.getCode();
453 switch (code) {
454 case 200:
455 return;
456 case 509:
457 try {
458 Thread.sleep(sleepTime);
459 } catch (InterruptedException e) { }
460 break;
461 default:
462 throw new IOException("delete request failed with " + code);
463 }
464 }
465 throw new IOException("delete request timed out");
466 }
467
468 public void delete(List<Delete> deletes) throws IOException {
469 for (Delete delete: deletes) {
470 delete(delete);
471 }
472 }
473
474 public void flushCommits() throws IOException {
475
476 }
477
478 class Scanner implements ResultScanner {
479
480 String uri;
481
482 public Scanner(Scan scan) throws IOException {
483 ScannerModel model;
484 try {
485 model = ScannerModel.fromScan(scan);
486 } catch (Exception e) {
487 throw new IOException(e);
488 }
489 StringBuffer sb = new StringBuffer();
490 sb.append('/');
491 sb.append(Bytes.toStringBinary(name));
492 sb.append('/');
493 sb.append("scanner");
494 for (int i = 0; i < maxRetries; i++) {
495 Response response = client.post(sb.toString(),
496 Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
497 int code = response.getCode();
498 switch (code) {
499 case 201:
500 uri = response.getLocation();
501 return;
502 case 509:
503 try {
504 Thread.sleep(sleepTime);
505 } catch (InterruptedException e) { }
506 break;
507 default:
508 throw new IOException("scan request failed with " + code);
509 }
510 }
511 throw new IOException("scan request timed out");
512 }
513
514 @Override
515 public Result[] next(int nbRows) throws IOException {
516 StringBuilder sb = new StringBuilder(uri);
517 sb.append("?n=");
518 sb.append(nbRows);
519 for (int i = 0; i < maxRetries; i++) {
520 Response response = client.get(sb.toString(),
521 Constants.MIMETYPE_PROTOBUF);
522 int code = response.getCode();
523 switch (code) {
524 case 200:
525 CellSetModel model = new CellSetModel();
526 model.getObjectFromMessage(response.getBody());
527 return buildResultFromModel(model);
528 case 204:
529 case 206:
530 return null;
531 case 509:
532 try {
533 Thread.sleep(sleepTime);
534 } catch (InterruptedException e) { }
535 break;
536 default:
537 throw new IOException("scanner.next request failed with " + code);
538 }
539 }
540 throw new IOException("scanner.next request timed out");
541 }
542
543 @Override
544 public Result next() throws IOException {
545 Result[] results = next(1);
546 if (results == null || results.length < 1) {
547 return null;
548 }
549 return results[0];
550 }
551
552 class Iter implements Iterator<Result> {
553
554 Result cache;
555
556 public Iter() {
557 try {
558 cache = Scanner.this.next();
559 } catch (IOException e) {
560 LOG.warn(StringUtils.stringifyException(e));
561 }
562 }
563
564 @Override
565 public boolean hasNext() {
566 return cache != null;
567 }
568
569 @Override
570 public Result next() {
571 Result result = cache;
572 try {
573 cache = Scanner.this.next();
574 } catch (IOException e) {
575 LOG.warn(StringUtils.stringifyException(e));
576 cache = null;
577 }
578 return result;
579 }
580
581 @Override
582 public void remove() {
583 throw new RuntimeException("remove() not supported");
584 }
585
586 }
587
588 @Override
589 public Iterator<Result> iterator() {
590 return new Iter();
591 }
592
593 @Override
594 public void close() {
595 try {
596 client.delete(uri);
597 } catch (IOException e) {
598 LOG.warn(StringUtils.stringifyException(e));
599 }
600 }
601
602 }
603
604 public ResultScanner getScanner(Scan scan) throws IOException {
605 return new Scanner(scan);
606 }
607
608 public ResultScanner getScanner(byte[] family) throws IOException {
609 Scan scan = new Scan();
610 scan.addFamily(family);
611 return new Scanner(scan);
612 }
613
614 public ResultScanner getScanner(byte[] family, byte[] qualifier)
615 throws IOException {
616 Scan scan = new Scan();
617 scan.addColumn(family, qualifier);
618 return new Scanner(scan);
619 }
620
621 public boolean isAutoFlush() {
622 return true;
623 }
624
625 public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
626 throw new IOException("getRowOrBefore not supported");
627 }
628
629 public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
630 byte[] value, Put put) throws IOException {
631
632 put.add(new KeyValue(row, family, qualifier, value));
633
634 CellSetModel model = buildModelFromPut(put);
635 StringBuilder sb = new StringBuilder();
636 sb.append('/');
637 sb.append(Bytes.toStringBinary(name));
638 sb.append('/');
639 sb.append(Bytes.toStringBinary(put.getRow()));
640 sb.append("?check=put");
641
642 for (int i = 0; i < maxRetries; i++) {
643 Response response = client.put(sb.toString(),
644 Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
645 int code = response.getCode();
646 switch (code) {
647 case 200:
648 return true;
649 case 304:
650 return false;
651 case 509:
652 try {
653 Thread.sleep(sleepTime);
654 } catch (final InterruptedException e) {
655 }
656 break;
657 default:
658 throw new IOException("checkAndPut request failed with " + code);
659 }
660 }
661 throw new IOException("checkAndPut request timed out");
662 }
663
664 public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
665 byte[] value, Delete delete) throws IOException {
666 Put put = new Put(row);
667
668 put.add(new KeyValue(row, family, qualifier, value));
669 CellSetModel model = buildModelFromPut(put);
670 StringBuilder sb = new StringBuilder();
671 sb.append('/');
672 sb.append(Bytes.toStringBinary(name));
673 sb.append('/');
674 sb.append(Bytes.toStringBinary(row));
675 sb.append("?check=delete");
676
677 for (int i = 0; i < maxRetries; i++) {
678 Response response = client.put(sb.toString(),
679 Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
680 int code = response.getCode();
681 switch (code) {
682 case 200:
683 return true;
684 case 304:
685 return false;
686 case 509:
687 try {
688 Thread.sleep(sleepTime);
689 } catch (final InterruptedException e) {
690 }
691 break;
692 default:
693 throw new IOException("checkAndDelete request failed with " + code);
694 }
695 }
696 throw new IOException("checkAndDelete request timed out");
697 }
698
699 public Result increment(Increment increment) throws IOException {
700 throw new IOException("Increment not supported");
701 }
702
703 public Result append(Append append) throws IOException {
704 throw new IOException("Append not supported");
705 }
706
707 public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
708 long amount) throws IOException {
709 throw new IOException("incrementColumnValue not supported");
710 }
711
712 public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
713 long amount, Durability durability) throws IOException {
714 throw new IOException("incrementColumnValue not supported");
715 }
716
717 @Override
718 public void batch(List<? extends Row> actions, Object[] results) throws IOException {
719 throw new IOException("batch not supported");
720 }
721
722 @Override
723 public Object[] batch(List<? extends Row> actions) throws IOException {
724 throw new IOException("batch not supported");
725 }
726
727 @Override
728 public <R> void batchCallback(List<? extends Row> actions, Object[] results,
729 Batch.Callback<R> callback) throws IOException, InterruptedException {
730 throw new IOException("batchCallback not supported");
731 }
732
733 @Override
734 public <R> Object[] batchCallback(List<? extends Row> actions, Batch.Callback<R> callback)
735 throws IOException, InterruptedException {
736 throw new IOException("batchCallback not supported");
737 }
738
739 @Override
740 public CoprocessorRpcChannel coprocessorService(byte[] row) {
741 throw new UnsupportedOperationException("coprocessorService not implemented");
742 }
743
744 @Override
745 public <T extends Service, R> Map<byte[], R> coprocessorService(Class<T> service,
746 byte[] startKey, byte[] endKey, Batch.Call<T, R> callable)
747 throws ServiceException, Throwable {
748 throw new UnsupportedOperationException("coprocessorService not implemented");
749 }
750
751 @Override
752 public <T extends Service, R> void coprocessorService(Class<T> service,
753 byte[] startKey, byte[] endKey, Batch.Call<T, R> callable, Batch.Callback<R> callback)
754 throws ServiceException, Throwable {
755 throw new UnsupportedOperationException("coprocessorService not implemented");
756 }
757
758 @Override
759 public void mutateRow(RowMutations rm) throws IOException {
760 throw new IOException("atomicMutation not supported");
761 }
762
763 @Override
764 public void setAutoFlush(boolean autoFlush) {
765 throw new UnsupportedOperationException("setAutoFlush not implemented");
766 }
767
768 @Override
769 public void setAutoFlush(boolean autoFlush, boolean clearBufferOnFail) {
770 throw new UnsupportedOperationException("setAutoFlush not implemented");
771 }
772
773 @Override
774 public long getWriteBufferSize() {
775 throw new UnsupportedOperationException("getWriteBufferSize not implemented");
776 }
777
778 @Override
779 public void setWriteBufferSize(long writeBufferSize) throws IOException {
780 throw new IOException("setWriteBufferSize not supported");
781 }
782 }