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