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