View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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   * HTable interface to remote tables accessed via REST gateway
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             // Puts use byte[] but Deletes use KeyValue
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    * Constructor
177    * @param client
178    * @param name
179    */
180   public RemoteHTable(Client client, String name) {
181     this(client, HBaseConfiguration.create(), Bytes.toBytes(name), null);
182   }
183 
184   /**
185    * Constructor
186    * @param client
187    * @param name
188    * @param accessToken
189    */
190   public RemoteHTable(Client client, String name, String accessToken) {
191     this(client, HBaseConfiguration.create(), Bytes.toBytes(name), accessToken);
192   }
193 
194   /**
195    * Constructor
196    * @param client
197    * @param conf
198    * @param name
199    * @param accessToken
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    * Constructor
208    * @param conf
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         // fall through
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     // this is a trick: The gateway accepts multiple rows in a cell set and
336     // ignores the row specification in the URI
337 
338     // separate puts by row
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     // build the cell set
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     // build path for multiput
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"); // can be any nonexistent row
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     // no-op
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 }