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