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.Delete;
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.Row;
51 import org.apache.hadoop.hbase.client.Result;
52 import org.apache.hadoop.hbase.client.ResultScanner;
53 import org.apache.hadoop.hbase.client.RowLock;
54 import org.apache.hadoop.hbase.client.Scan;
55 import org.apache.hadoop.hbase.io.TimeRange;
56 import org.apache.hadoop.hbase.rest.Constants;
57 import org.apache.hadoop.hbase.rest.model.CellModel;
58 import org.apache.hadoop.hbase.rest.model.CellSetModel;
59 import org.apache.hadoop.hbase.rest.model.RowModel;
60 import org.apache.hadoop.hbase.rest.model.ScannerModel;
61 import org.apache.hadoop.hbase.rest.model.TableSchemaModel;
62 import org.apache.hadoop.hbase.util.Bytes;
63
64
65
66
67 public class RemoteHTable implements HTableInterface {
68
69 private static final Log LOG = LogFactory.getLog(RemoteHTable.class);
70
71 final Client client;
72 final Configuration conf;
73 final byte[] name;
74 final String accessToken;
75 final int maxRetries;
76 final long sleepTime;
77
78 @SuppressWarnings("unchecked")
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 if (accessToken != null) {
84 sb.append(accessToken);
85 sb.append('/');
86 }
87 sb.append(Bytes.toStringBinary(name));
88 sb.append('/');
89 sb.append(Bytes.toStringBinary(row));
90 Set families = familyMap.entrySet();
91 if (families != null) {
92 Iterator i = familyMap.entrySet().iterator();
93 if (i.hasNext()) {
94 sb.append('/');
95 }
96 while (i.hasNext()) {
97 Map.Entry e = (Map.Entry)i.next();
98 Collection quals = (Collection)e.getValue();
99 if (quals != null && !quals.isEmpty()) {
100 Iterator ii = quals.iterator();
101 while (ii.hasNext()) {
102 sb.append(Bytes.toStringBinary((byte[])e.getKey()));
103 sb.append(':');
104 Object o = ii.next();
105
106 if (o instanceof byte[]) {
107 sb.append(Bytes.toStringBinary((byte[])o));
108 } else if (o instanceof KeyValue) {
109 sb.append(Bytes.toStringBinary(((KeyValue)o).getQualifier()));
110 } else {
111 throw new RuntimeException("object type not handled");
112 }
113 if (ii.hasNext()) {
114 sb.append(',');
115 }
116 }
117 } else {
118 sb.append(Bytes.toStringBinary((byte[])e.getKey()));
119 sb.append(':');
120 }
121 if (i.hasNext()) {
122 sb.append(',');
123 }
124 }
125 }
126 if (startTime != 0 && endTime != Long.MAX_VALUE) {
127 sb.append('/');
128 sb.append(startTime);
129 if (startTime != endTime) {
130 sb.append(',');
131 sb.append(endTime);
132 }
133 } else if (endTime != Long.MAX_VALUE) {
134 sb.append('/');
135 sb.append(endTime);
136 }
137 if (maxVersions > 1) {
138 sb.append("?v=");
139 sb.append(maxVersions);
140 }
141 return sb.toString();
142 }
143
144 protected Result[] buildResultFromModel(final CellSetModel model) {
145 List<Result> results = new ArrayList<Result>();
146 for (RowModel row: model.getRows()) {
147 List<KeyValue> kvs = new ArrayList<KeyValue>();
148 for (CellModel cell: row.getCells()) {
149 byte[][] split = KeyValue.parseColumn(cell.getColumn());
150 byte[] column = split[0];
151 byte[] qualifier = split.length > 1 ? split[1] : null;
152 kvs.add(new KeyValue(row.getKey(), column, qualifier,
153 cell.getTimestamp(), cell.getValue()));
154 }
155 results.add(new Result(kvs));
156 }
157 return results.toArray(new Result[results.size()]);
158 }
159
160 protected CellSetModel buildModelFromPut(Put put) {
161 RowModel row = new RowModel(put.getRow());
162 long ts = put.getTimeStamp();
163 for (List<KeyValue> kvs: put.getFamilyMap().values()) {
164 for (KeyValue kv: kvs) {
165 row.addCell(new CellModel(kv.getFamily(), kv.getQualifier(),
166 ts != HConstants.LATEST_TIMESTAMP ? ts : kv.getTimestamp(),
167 kv.getValue()));
168 }
169 }
170 CellSetModel model = new CellSetModel();
171 model.addRow(row);
172 return model;
173 }
174
175
176
177
178
179
180 public RemoteHTable(Client client, String name) {
181 this(client, HBaseConfiguration.create(), Bytes.toBytes(name), null);
182 }
183
184
185
186
187
188
189
190 public RemoteHTable(Client client, String name, String accessToken) {
191 this(client, HBaseConfiguration.create(), Bytes.toBytes(name), accessToken);
192 }
193
194
195
196
197
198
199
200
201 public RemoteHTable(Client client, Configuration conf, String name,
202 String accessToken) {
203 this(client, conf, Bytes.toBytes(name), accessToken);
204 }
205
206
207
208
209
210 public RemoteHTable(Client client, Configuration conf, byte[] name,
211 String accessToken) {
212 this.client = client;
213 this.conf = conf;
214 this.name = name;
215 this.accessToken = accessToken;
216 this.maxRetries = conf.getInt("hbase.rest.client.max.retries", 10);
217 this.sleepTime = conf.getLong("hbase.rest.client.sleep", 1000);
218 }
219
220 public byte[] getTableName() {
221 return name.clone();
222 }
223
224 public Configuration getConfiguration() {
225 return conf;
226 }
227
228 public HTableDescriptor getTableDescriptor() throws IOException {
229 StringBuilder sb = new StringBuilder();
230 sb.append('/');
231 if (accessToken != null) {
232 sb.append(accessToken);
233 sb.append('/');
234 }
235 sb.append(Bytes.toStringBinary(name));
236 sb.append('/');
237 sb.append("schema");
238 for (int i = 0; i < maxRetries; i++) {
239 Response response = client.get(sb.toString(), Constants.MIMETYPE_PROTOBUF);
240 int code = response.getCode();
241 switch (code) {
242 case 200:
243 TableSchemaModel schema = new TableSchemaModel();
244 schema.getObjectFromMessage(response.getBody());
245 return schema.getTableDescriptor();
246 case 509:
247 try {
248 Thread.sleep(sleepTime);
249 } catch (InterruptedException e) { }
250 break;
251 default:
252 throw new IOException("schema request returned " + code);
253 }
254 }
255 throw new IOException("schema request timed out");
256 }
257
258 public void close() throws IOException {
259 client.shutdown();
260 }
261
262 public Result get(Get get) throws IOException {
263 TimeRange range = get.getTimeRange();
264 String spec = buildRowSpec(get.getRow(), get.getFamilyMap(),
265 range.getMin(), range.getMax(), get.getMaxVersions());
266 if (get.getFilter() != null) {
267 LOG.warn("filters not supported on gets");
268 }
269 for (int i = 0; i < maxRetries; i++) {
270 Response response = client.get(spec, Constants.MIMETYPE_PROTOBUF);
271 int code = response.getCode();
272 switch (code) {
273 case 200:
274 CellSetModel model = new CellSetModel();
275 model.getObjectFromMessage(response.getBody());
276 Result[] results = buildResultFromModel(model);
277 if (results.length > 0) {
278 if (results.length > 1) {
279 LOG.warn("too many results for get (" + results.length + ")");
280 }
281 return results[0];
282 }
283
284 case 404:
285 return new Result();
286 case 509:
287 try {
288 Thread.sleep(sleepTime);
289 } catch (InterruptedException e) { }
290 break;
291 default:
292 throw new IOException("get request returned " + code);
293 }
294 }
295 throw new IOException("get request timed out");
296 }
297
298 public boolean exists(Get get) throws IOException {
299 LOG.warn("exists() is really get(), just use get()");
300 Result result = get(get);
301 return (result != null && !(result.isEmpty()));
302 }
303
304 public void put(Put put) throws IOException {
305 CellSetModel model = buildModelFromPut(put);
306 StringBuilder sb = new StringBuilder();
307 sb.append('/');
308 if (accessToken != null) {
309 sb.append(accessToken);
310 sb.append('/');
311 }
312 sb.append(Bytes.toStringBinary(name));
313 sb.append('/');
314 sb.append(Bytes.toStringBinary(put.getRow()));
315 for (int i = 0; i < maxRetries; i++) {
316 Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
317 model.createProtobufOutput());
318 int code = response.getCode();
319 switch (code) {
320 case 200:
321 return;
322 case 509:
323 try {
324 Thread.sleep(sleepTime);
325 } catch (InterruptedException e) { }
326 break;
327 default:
328 throw new IOException("put request failed with " + code);
329 }
330 }
331 throw new IOException("put request timed out");
332 }
333
334 public void put(List<Put> puts) throws IOException {
335
336
337
338
339 TreeMap<byte[],List<KeyValue>> map =
340 new TreeMap<byte[],List<KeyValue>>(Bytes.BYTES_COMPARATOR);
341 for (Put put: puts) {
342 byte[] row = put.getRow();
343 List<KeyValue> kvs = map.get(row);
344 if (kvs == null) {
345 kvs = new ArrayList<KeyValue>();
346 map.put(row, kvs);
347 }
348 for (List<KeyValue> l: put.getFamilyMap().values()) {
349 kvs.addAll(l);
350 }
351 }
352
353
354 CellSetModel model = new CellSetModel();
355 for (Map.Entry<byte[], List<KeyValue>> e: map.entrySet()) {
356 RowModel row = new RowModel(e.getKey());
357 for (KeyValue kv: e.getValue()) {
358 row.addCell(new CellModel(kv));
359 }
360 model.addRow(row);
361 }
362
363
364 StringBuilder sb = new StringBuilder();
365 sb.append('/');
366 if (accessToken != null) {
367 sb.append(accessToken);
368 sb.append('/');
369 }
370 sb.append(Bytes.toStringBinary(name));
371 sb.append("/$multiput");
372 for (int i = 0; i < maxRetries; i++) {
373 Response response = client.put(sb.toString(), Constants.MIMETYPE_PROTOBUF,
374 model.createProtobufOutput());
375 int code = response.getCode();
376 switch (code) {
377 case 200:
378 return;
379 case 509:
380 try {
381 Thread.sleep(sleepTime);
382 } catch (InterruptedException e) { }
383 break;
384 default:
385 throw new IOException("multiput request failed with " + code);
386 }
387 }
388 throw new IOException("multiput request timed out");
389 }
390
391 public void delete(Delete delete) throws IOException {
392 String spec = buildRowSpec(delete.getRow(), delete.getFamilyMap(),
393 delete.getTimeStamp(), delete.getTimeStamp(), 1);
394 for (int i = 0; i < maxRetries; i++) {
395 Response response = client.delete(spec);
396 int code = response.getCode();
397 switch (code) {
398 case 200:
399 return;
400 case 509:
401 try {
402 Thread.sleep(sleepTime);
403 } catch (InterruptedException e) { }
404 break;
405 default:
406 throw new IOException("delete request failed with " + code);
407 }
408 }
409 throw new IOException("delete request timed out");
410 }
411
412 public void delete(List<Delete> deletes) throws IOException {
413 for (Delete delete: deletes) {
414 delete(delete);
415 }
416 }
417
418 public void flushCommits() throws IOException {
419
420 }
421
422 class Scanner implements ResultScanner {
423
424 String uri;
425
426 public Scanner(Scan scan) throws IOException {
427 ScannerModel model;
428 try {
429 model = ScannerModel.fromScan(scan);
430 } catch (Exception e) {
431 throw new IOException(e);
432 }
433 StringBuffer sb = new StringBuffer();
434 sb.append('/');
435 if (accessToken != null) {
436 sb.append(accessToken);
437 sb.append('/');
438 }
439 sb.append(Bytes.toStringBinary(name));
440 sb.append('/');
441 sb.append("scanner");
442 for (int i = 0; i < maxRetries; i++) {
443 Response response = client.post(sb.toString(),
444 Constants.MIMETYPE_PROTOBUF, model.createProtobufOutput());
445 int code = response.getCode();
446 switch (code) {
447 case 201:
448 uri = response.getLocation();
449 return;
450 case 509:
451 try {
452 Thread.sleep(sleepTime);
453 } catch (InterruptedException e) { }
454 break;
455 default:
456 throw new IOException("scan request failed with " + code);
457 }
458 }
459 throw new IOException("scan request timed out");
460 }
461
462 @Override
463 public Result[] next(int nbRows) throws IOException {
464 StringBuilder sb = new StringBuilder(uri);
465 sb.append("?n=");
466 sb.append(nbRows);
467 for (int i = 0; i < maxRetries; i++) {
468 Response response = client.get(sb.toString(),
469 Constants.MIMETYPE_PROTOBUF);
470 int code = response.getCode();
471 switch (code) {
472 case 200:
473 CellSetModel model = new CellSetModel();
474 model.getObjectFromMessage(response.getBody());
475 return buildResultFromModel(model);
476 case 204:
477 case 206:
478 return null;
479 case 509:
480 try {
481 Thread.sleep(sleepTime);
482 } catch (InterruptedException e) { }
483 break;
484 default:
485 throw new IOException("scanner.next request failed with " + code);
486 }
487 }
488 throw new IOException("scanner.next request timed out");
489 }
490
491 @Override
492 public Result next() throws IOException {
493 Result[] results = next(1);
494 if (results == null || results.length < 1) {
495 return null;
496 }
497 return results[0];
498 }
499
500 class Iter implements Iterator<Result> {
501
502 Result cache;
503
504 public Iter() {
505 try {
506 cache = Scanner.this.next();
507 } catch (IOException e) {
508 LOG.warn(StringUtils.stringifyException(e));
509 }
510 }
511
512 @Override
513 public boolean hasNext() {
514 return cache != null;
515 }
516
517 @Override
518 public Result next() {
519 Result result = cache;
520 try {
521 cache = Scanner.this.next();
522 } catch (IOException e) {
523 LOG.warn(StringUtils.stringifyException(e));
524 cache = null;
525 }
526 return result;
527 }
528
529 @Override
530 public void remove() {
531 throw new RuntimeException("remove() not supported");
532 }
533
534 }
535
536 @Override
537 public Iterator<Result> iterator() {
538 return new Iter();
539 }
540
541 @Override
542 public void close() {
543 try {
544 client.delete(uri);
545 } catch (IOException e) {
546 LOG.warn(StringUtils.stringifyException(e));
547 }
548 }
549
550 }
551
552 public ResultScanner getScanner(Scan scan) throws IOException {
553 return new Scanner(scan);
554 }
555
556 public ResultScanner getScanner(byte[] family) throws IOException {
557 Scan scan = new Scan();
558 scan.addFamily(family);
559 return new Scanner(scan);
560 }
561
562 public ResultScanner getScanner(byte[] family, byte[] qualifier)
563 throws IOException {
564 Scan scan = new Scan();
565 scan.addColumn(family, qualifier);
566 return new Scanner(scan);
567 }
568
569 public boolean isAutoFlush() {
570 return true;
571 }
572
573 public Result getRowOrBefore(byte[] row, byte[] family) throws IOException {
574 throw new IOException("getRowOrBefore not supported");
575 }
576
577 public RowLock lockRow(byte[] row) throws IOException {
578 throw new IOException("lockRow not implemented");
579 }
580
581 public void unlockRow(RowLock rl) throws IOException {
582 throw new IOException("unlockRow not implemented");
583 }
584
585 public boolean checkAndPut(byte[] row, byte[] family, byte[] qualifier,
586 byte[] value, Put put) throws IOException {
587 throw new IOException("checkAndPut not supported");
588 }
589
590 public boolean checkAndDelete(byte[] row, byte[] family, byte[] qualifier,
591 byte[] value, Delete delete) throws IOException {
592 throw new IOException("checkAndDelete not supported");
593 }
594
595 public Result increment(Increment increment) throws IOException {
596 throw new IOException("Increment not supported");
597 }
598
599 public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
600 long amount) throws IOException {
601 throw new IOException("incrementColumnValue not supported");
602 }
603
604 public long incrementColumnValue(byte[] row, byte[] family, byte[] qualifier,
605 long amount, boolean writeToWAL) throws IOException {
606 throw new IOException("incrementColumnValue not supported");
607 }
608
609 @Override
610 public void batch(List<Row> actions, Object[] results) throws IOException {
611 throw new IOException("batch not supported");
612 }
613
614 @Override
615 public Object[] batch(List<Row> actions) throws IOException {
616 throw new IOException("batch not supported");
617 }
618
619 @Override
620 public Result[] get(List<Get> gets) throws IOException {
621 throw new IOException("get(List<Get>) not supported");
622 }
623
624 @Override
625 public <T extends CoprocessorProtocol> T coprocessorProxy(Class<T> protocol,
626 byte[] row) {
627 throw new
628 UnsupportedOperationException("coprocessorProxy not implemented");
629 }
630
631 @Override
632 public <T extends CoprocessorProtocol, R> Map<byte[], R> coprocessorExec(
633 Class<T> protocol, byte[] startKey, byte[] endKey,
634 Batch.Call<T, R> callable)
635 throws IOException, Throwable {
636 throw new UnsupportedOperationException("coprocessorExec not implemented");
637 }
638
639 @Override
640 public <T extends CoprocessorProtocol, R> void coprocessorExec(
641 Class<T> protocol, byte[] startKey, byte[] endKey,
642 Batch.Call<T, R> callable, Batch.Callback<R> callback)
643 throws IOException, Throwable {
644 throw new UnsupportedOperationException("coprocessorExec not implemented");
645 }
646 }