View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.thrift;
20  
21  import java.io.IOException;
22  import java.net.InetAddress;
23  import java.net.InetSocketAddress;
24  import java.net.UnknownHostException;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.HashMap;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.TreeMap;
31  
32  import org.apache.commons.cli.CommandLine;
33  import org.apache.commons.cli.CommandLineParser;
34  import org.apache.commons.cli.HelpFormatter;
35  import org.apache.commons.cli.Option;
36  import org.apache.commons.cli.OptionGroup;
37  import org.apache.commons.cli.Options;
38  import org.apache.commons.cli.PosixParser;
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.hbase.HBaseConfiguration;
43  import org.apache.hadoop.hbase.HColumnDescriptor;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HRegionInfo;
46  import org.apache.hadoop.hbase.HServerAddress;
47  import org.apache.hadoop.hbase.HTableDescriptor;
48  import org.apache.hadoop.hbase.KeyValue;
49  import org.apache.hadoop.hbase.client.Delete;
50  import org.apache.hadoop.hbase.client.Get;
51  import org.apache.hadoop.hbase.client.HBaseAdmin;
52  import org.apache.hadoop.hbase.client.HTable;
53  import org.apache.hadoop.hbase.client.Put;
54  import org.apache.hadoop.hbase.client.Result;
55  import org.apache.hadoop.hbase.client.ResultScanner;
56  import org.apache.hadoop.hbase.client.Scan;
57  import org.apache.hadoop.hbase.filter.Filter;
58  import org.apache.hadoop.hbase.filter.PrefixFilter;
59  import org.apache.hadoop.hbase.filter.WhileMatchFilter;
60  import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
61  import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
62  import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
63  import org.apache.hadoop.hbase.thrift.generated.Hbase;
64  import org.apache.hadoop.hbase.thrift.generated.IOError;
65  import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
66  import org.apache.hadoop.hbase.thrift.generated.Mutation;
67  import org.apache.hadoop.hbase.thrift.generated.TCell;
68  import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
69  import org.apache.hadoop.hbase.thrift.generated.TRowResult;
70  import org.apache.hadoop.hbase.util.Bytes;
71  import org.apache.thrift.TException;
72  import org.apache.thrift.protocol.TBinaryProtocol;
73  import org.apache.thrift.protocol.TCompactProtocol;
74  import org.apache.thrift.protocol.TProtocolFactory;
75  import org.apache.thrift.server.THsHaServer;
76  import org.apache.thrift.server.TNonblockingServer;
77  import org.apache.thrift.server.TServer;
78  import org.apache.thrift.server.TThreadPoolServer;
79  import org.apache.thrift.transport.TFramedTransport;
80  import org.apache.thrift.transport.TNonblockingServerSocket;
81  import org.apache.thrift.transport.TNonblockingServerTransport;
82  import org.apache.thrift.transport.TServerSocket;
83  import org.apache.thrift.transport.TServerTransport;
84  import org.apache.thrift.transport.TTransportFactory;
85  
86  /**
87   * ThriftServer - this class starts up a Thrift server which implements the
88   * Hbase API specified in the Hbase.thrift IDL file.
89   */
90  public class ThriftServer {
91  
92    /**
93     * The HBaseHandler is a glue object that connects Thrift RPC calls to the
94     * HBase client API primarily defined in the HBaseAdmin and HTable objects.
95     */
96    public static class HBaseHandler implements Hbase.Iface {
97      protected Configuration conf;
98      protected HBaseAdmin admin = null;
99      protected final Log LOG = LogFactory.getLog(this.getClass().getName());
100 
101     // nextScannerId and scannerMap are used to manage scanner state
102     protected int nextScannerId = 0;
103     protected HashMap<Integer, ResultScanner> scannerMap = null;
104 
105     private static ThreadLocal<Map<String, HTable>> threadLocalTables = new ThreadLocal<Map<String, HTable>>() {
106       @Override
107       protected Map<String, HTable> initialValue() {
108         return new TreeMap<String, HTable>();
109       }
110     };
111 
112     /**
113      * Returns a list of all the column families for a given htable.
114      *
115      * @param table
116      * @return
117      * @throws IOException
118      */
119     byte[][] getAllColumns(HTable table) throws IOException {
120       HColumnDescriptor[] cds = table.getTableDescriptor().getColumnFamilies();
121       byte[][] columns = new byte[cds.length][];
122       for (int i = 0; i < cds.length; i++) {
123         columns[i] = Bytes.add(cds[i].getName(),
124             KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
125       }
126       return columns;
127     }
128 
129     /**
130      * Creates and returns an HTable instance from a given table name.
131      *
132      * @param tableName
133      *          name of table
134      * @return HTable object
135      * @throws IOException
136      * @throws IOError
137      */
138     protected HTable getTable(final byte[] tableName) throws IOError,
139         IOException {
140       String table = new String(tableName);
141       Map<String, HTable> tables = threadLocalTables.get();
142       if (!tables.containsKey(table)) {
143         tables.put(table, new HTable(conf, tableName));
144       }
145       return tables.get(table);
146     }
147 
148     /**
149      * Assigns a unique ID to the scanner and adds the mapping to an internal
150      * hash-map.
151      *
152      * @param scanner
153      * @return integer scanner id
154      */
155     protected synchronized int addScanner(ResultScanner scanner) {
156       int id = nextScannerId++;
157       scannerMap.put(id, scanner);
158       return id;
159     }
160 
161     /**
162      * Returns the scanner associated with the specified ID.
163      *
164      * @param id
165      * @return a Scanner, or null if ID was invalid.
166      */
167     protected synchronized ResultScanner getScanner(int id) {
168       return scannerMap.get(id);
169     }
170 
171     /**
172      * Removes the scanner associated with the specified ID from the internal
173      * id->scanner hash-map.
174      *
175      * @param id
176      * @return a Scanner, or null if ID was invalid.
177      */
178     protected synchronized ResultScanner removeScanner(int id) {
179       return scannerMap.remove(id);
180     }
181 
182     /**
183      * Constructs an HBaseHandler object.
184      * @throws IOException 
185      */
186     HBaseHandler()
187     throws IOException {
188       this(HBaseConfiguration.create());
189     }
190 
191     HBaseHandler(final Configuration c)
192     throws IOException {
193       this.conf = c;
194       admin = new HBaseAdmin(conf);
195       scannerMap = new HashMap<Integer, ResultScanner>();
196     }
197 
198     public void enableTable(final byte[] tableName) throws IOError {
199       try{
200         admin.enableTable(tableName);
201       } catch (IOException e) {
202         throw new IOError(e.getMessage());
203       }
204     }
205 
206     public void disableTable(final byte[] tableName) throws IOError{
207       try{
208         admin.disableTable(tableName);
209       } catch (IOException e) {
210         throw new IOError(e.getMessage());
211       }
212     }
213 
214     public boolean isTableEnabled(final byte[] tableName) throws IOError {
215       try {
216         return HTable.isTableEnabled(this.conf, tableName);
217       } catch (IOException e) {
218         throw new IOError(e.getMessage());
219       }
220     }
221 
222     public void compact(byte[] tableNameOrRegionName) throws IOError {
223       try{
224         admin.compact(tableNameOrRegionName);
225       } catch (InterruptedException e) {
226         throw new IOError(e.getMessage());
227       } catch (IOException e) {
228         throw new IOError(e.getMessage());
229       }
230     }
231 
232     public void majorCompact(byte[] tableNameOrRegionName) throws IOError {
233       try{
234         admin.majorCompact(tableNameOrRegionName);
235       } catch (InterruptedException e) {
236         throw new IOError(e.getMessage());
237       } catch (IOException e) {
238         throw new IOError(e.getMessage());
239       }
240     }
241 
242     public List<byte[]> getTableNames() throws IOError {
243       try {
244         HTableDescriptor[] tables = this.admin.listTables();
245         ArrayList<byte[]> list = new ArrayList<byte[]>(tables.length);
246         for (int i = 0; i < tables.length; i++) {
247           list.add(tables[i].getName());
248         }
249         return list;
250       } catch (IOException e) {
251         throw new IOError(e.getMessage());
252       }
253     }
254 
255     public List<TRegionInfo> getTableRegions(byte[] tableName)
256     throws IOError {
257       try{
258         HTable table = getTable(tableName);
259         Map<HRegionInfo, HServerAddress> regionsInfo = table.getRegionsInfo();
260         List<TRegionInfo> regions = new ArrayList<TRegionInfo>();
261 
262         for (HRegionInfo regionInfo : regionsInfo.keySet()){
263           TRegionInfo region = new TRegionInfo();
264           region.startKey = regionInfo.getStartKey();
265           region.endKey = regionInfo.getEndKey();
266           region.id = regionInfo.getRegionId();
267           region.name = regionInfo.getRegionName();
268           region.version = regionInfo.getVersion();
269           regions.add(region);
270         }
271         return regions;
272       } catch (IOException e){
273         throw new IOError(e.getMessage());
274       }
275     }
276 
277     @Deprecated
278     public List<TCell> get(byte[] tableName, byte[] row, byte[] column)
279         throws IOError {
280       byte [][] famAndQf = KeyValue.parseColumn(column);
281       if(famAndQf.length == 1) {
282         return get(tableName, row, famAndQf[0], new byte[0]);
283       }
284       return get(tableName, row, famAndQf[0], famAndQf[1]);
285     }
286 
287     public List<TCell> get(byte [] tableName, byte [] row, byte [] family,
288         byte [] qualifier) throws IOError {
289       try {
290         HTable table = getTable(tableName);
291         Get get = new Get(row);
292         if (qualifier == null || qualifier.length == 0) {
293           get.addFamily(family);
294         } else {
295           get.addColumn(family, qualifier);
296         }
297         Result result = table.get(get);
298         return ThriftUtilities.cellFromHBase(result.sorted());
299       } catch (IOException e) {
300         throw new IOError(e.getMessage());
301       }
302     }
303 
304     @Deprecated
305     public List<TCell> getVer(byte[] tableName, byte[] row,
306         byte[] column, int numVersions) throws IOError {
307       byte [][] famAndQf = KeyValue.parseColumn(column);
308       if(famAndQf.length == 1) {
309         return getVer(tableName, row, famAndQf[0], new byte[0], numVersions);
310       }
311       return getVer(tableName, row, famAndQf[0], famAndQf[1], numVersions);
312     }
313 
314     public List<TCell> getVer(byte [] tableName, byte [] row, byte [] family,
315         byte [] qualifier, int numVersions) throws IOError {
316       try {
317         HTable table = getTable(tableName);
318         Get get = new Get(row);
319         get.addColumn(family, qualifier);
320         get.setMaxVersions(numVersions);
321         Result result = table.get(get);
322         return ThriftUtilities.cellFromHBase(result.sorted());
323       } catch (IOException e) {
324         throw new IOError(e.getMessage());
325       }
326     }
327 
328     @Deprecated
329     public List<TCell> getVerTs(byte[] tableName, byte[] row,
330         byte[] column, long timestamp, int numVersions) throws IOError {
331       byte [][] famAndQf = KeyValue.parseColumn(column);
332       if(famAndQf.length == 1) {
333         return getVerTs(tableName, row, famAndQf[0], new byte[0], timestamp,
334             numVersions);
335       }
336       return getVerTs(tableName, row, famAndQf[0], famAndQf[1], timestamp,
337           numVersions);
338     }
339 
340     public List<TCell> getVerTs(byte [] tableName, byte [] row, byte [] family,
341         byte [] qualifier, long timestamp, int numVersions) throws IOError {
342       try {
343         HTable table = getTable(tableName);
344         Get get = new Get(row);
345         get.addColumn(family, qualifier);
346         get.setTimeRange(Long.MIN_VALUE, timestamp);
347         get.setMaxVersions(numVersions);
348         Result result = table.get(get);
349         return ThriftUtilities.cellFromHBase(result.sorted());
350       } catch (IOException e) {
351         throw new IOError(e.getMessage());
352       }
353     }
354 
355     public List<TRowResult> getRow(byte[] tableName, byte[] row)
356         throws IOError {
357       return getRowWithColumnsTs(tableName, row, null,
358                                  HConstants.LATEST_TIMESTAMP);
359     }
360 
361     public List<TRowResult> getRowWithColumns(byte[] tableName, byte[] row,
362         List<byte[]> columns) throws IOError {
363       return getRowWithColumnsTs(tableName, row, columns,
364                                  HConstants.LATEST_TIMESTAMP);
365     }
366 
367     public List<TRowResult> getRowTs(byte[] tableName, byte[] row,
368         long timestamp) throws IOError {
369       return getRowWithColumnsTs(tableName, row, null,
370                                  timestamp);
371     }
372 
373     public List<TRowResult> getRowWithColumnsTs(byte[] tableName, byte[] row,
374         List<byte[]> columns, long timestamp) throws IOError {
375       try {
376         HTable table = getTable(tableName);
377         if (columns == null) {
378           Get get = new Get(row);
379           get.setTimeRange(Long.MIN_VALUE, timestamp);
380           Result result = table.get(get);
381           return ThriftUtilities.rowResultFromHBase(result);
382         }
383         byte[][] columnArr = columns.toArray(new byte[columns.size()][]);
384         Get get = new Get(row);
385         for(byte [] column : columnArr) {
386           byte [][] famAndQf = KeyValue.parseColumn(column);
387           if (famAndQf.length == 1) {
388               get.addFamily(famAndQf[0]);
389           } else {
390               get.addColumn(famAndQf[0], famAndQf[1]);
391           }
392         }
393         get.setTimeRange(Long.MIN_VALUE, timestamp);
394         Result result = table.get(get);
395         return ThriftUtilities.rowResultFromHBase(result);
396       } catch (IOException e) {
397         throw new IOError(e.getMessage());
398       }
399     }
400 
401     public void deleteAll(byte[] tableName, byte[] row, byte[] column)
402         throws IOError {
403       deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP);
404     }
405 
406     public void deleteAllTs(byte[] tableName, byte[] row, byte[] column,
407         long timestamp) throws IOError {
408       try {
409         HTable table = getTable(tableName);
410         Delete delete  = new Delete(row);
411         byte [][] famAndQf = KeyValue.parseColumn(column);
412         if (famAndQf.length == 1) {
413           delete.deleteFamily(famAndQf[0], timestamp);
414         } else {
415           delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
416         }
417         table.delete(delete);
418 
419       } catch (IOException e) {
420         throw new IOError(e.getMessage());
421       }
422     }
423 
424     public void deleteAllRow(byte[] tableName, byte[] row) throws IOError {
425       deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP);
426     }
427 
428     public void deleteAllRowTs(byte[] tableName, byte[] row, long timestamp)
429         throws IOError {
430       try {
431         HTable table = getTable(tableName);
432         Delete delete  = new Delete(row, timestamp, null);
433         table.delete(delete);
434       } catch (IOException e) {
435         throw new IOError(e.getMessage());
436       }
437     }
438 
439     public void createTable(byte[] tableName,
440         List<ColumnDescriptor> columnFamilies) throws IOError,
441         IllegalArgument, AlreadyExists {
442       try {
443         if (admin.tableExists(tableName)) {
444           throw new AlreadyExists("table name already in use");
445         }
446         HTableDescriptor desc = new HTableDescriptor(tableName);
447         for (ColumnDescriptor col : columnFamilies) {
448           HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
449           desc.addFamily(colDesc);
450         }
451         admin.createTable(desc);
452       } catch (IOException e) {
453         throw new IOError(e.getMessage());
454       } catch (IllegalArgumentException e) {
455         throw new IllegalArgument(e.getMessage());
456       }
457     }
458 
459     public void deleteTable(byte[] tableName) throws IOError {
460       if (LOG.isDebugEnabled()) {
461         LOG.debug("deleteTable: table=" + new String(tableName));
462       }
463       try {
464         if (!admin.tableExists(tableName)) {
465           throw new IOError("table does not exist");
466         }
467         admin.deleteTable(tableName);
468       } catch (IOException e) {
469         throw new IOError(e.getMessage());
470       }
471     }
472 
473     public void mutateRow(byte[] tableName, byte[] row,
474         List<Mutation> mutations) throws IOError, IllegalArgument {
475       mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP);
476     }
477 
478     public void mutateRowTs(byte[] tableName, byte[] row,
479         List<Mutation> mutations, long timestamp) throws IOError, IllegalArgument {
480       HTable table = null;
481       try {
482         table = getTable(tableName);
483         Put put = new Put(row, timestamp, null);
484 
485         Delete delete = new Delete(row);
486 
487         // I apologize for all this mess :)
488         for (Mutation m : mutations) {
489           byte[][] famAndQf = KeyValue.parseColumn(m.column);
490           if (m.isDelete) {
491             if (famAndQf.length == 1) {
492               delete.deleteFamily(famAndQf[0], timestamp);
493             } else {
494               delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
495             }
496           } else {
497             if(famAndQf.length == 1) {
498               put.add(famAndQf[0], new byte[0], m.value);
499             } else {
500               put.add(famAndQf[0], famAndQf[1], m.value);
501             }
502           }
503         }
504         if (!delete.isEmpty())
505           table.delete(delete);
506         if (!put.isEmpty())
507           table.put(put);
508       } catch (IOException e) {
509         throw new IOError(e.getMessage());
510       } catch (IllegalArgumentException e) {
511         throw new IllegalArgument(e.getMessage());
512       }
513     }
514 
515     public void mutateRows(byte[] tableName, List<BatchMutation> rowBatches)
516         throws IOError, IllegalArgument, TException {
517       mutateRowsTs(tableName, rowBatches, HConstants.LATEST_TIMESTAMP);
518     }
519 
520     public void mutateRowsTs(byte[] tableName, List<BatchMutation> rowBatches, long timestamp)
521         throws IOError, IllegalArgument, TException {
522       List<Put> puts = new ArrayList<Put>();
523       List<Delete> deletes = new ArrayList<Delete>();
524 
525       for (BatchMutation batch : rowBatches) {
526         byte[] row = batch.row;
527         List<Mutation> mutations = batch.mutations;
528         Delete delete = new Delete(row);
529         Put put = new Put(row, timestamp, null);
530         for (Mutation m : mutations) {
531           byte[][] famAndQf = KeyValue.parseColumn(m.column);
532           if (m.isDelete) {
533             // no qualifier, family only.
534             if (famAndQf.length == 1) {
535               delete.deleteFamily(famAndQf[0], timestamp);
536             } else {
537               delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
538             }
539           } else {
540             if(famAndQf.length == 1) {
541               put.add(famAndQf[0], new byte[0], m.value);
542             } else {
543               put.add(famAndQf[0], famAndQf[1], m.value);
544             }
545           }
546         }
547         if (!delete.isEmpty())
548           deletes.add(delete);
549         if (!put.isEmpty())
550           puts.add(put);
551       }
552 
553       HTable table = null;
554       try {
555         table = getTable(tableName);
556         if (!puts.isEmpty())
557           table.put(puts);
558         for (Delete del : deletes) {
559           table.delete(del);
560         }
561       } catch (IOException e) {
562         throw new IOError(e.getMessage());
563       } catch (IllegalArgumentException e) {
564         throw new IllegalArgument(e.getMessage());
565       }
566     }
567 
568     @Deprecated
569     public long atomicIncrement(byte[] tableName, byte[] row, byte[] column,
570         long amount) throws IOError, IllegalArgument, TException {
571       byte [][] famAndQf = KeyValue.parseColumn(column);
572       if(famAndQf.length == 1) {
573         return atomicIncrement(tableName, row, famAndQf[0], new byte[0],
574             amount);
575       }
576       return atomicIncrement(tableName, row, famAndQf[0], famAndQf[1], amount);
577     }
578 
579     public long atomicIncrement(byte [] tableName, byte [] row, byte [] family,
580         byte [] qualifier, long amount)
581     throws IOError, IllegalArgument, TException {
582       HTable table;
583       try {
584         table = getTable(tableName);
585         return table.incrementColumnValue(row, family, qualifier, amount);
586       } catch (IOException e) {
587         throw new IOError(e.getMessage());
588       }
589     }
590 
591     public void scannerClose(int id) throws IOError, IllegalArgument {
592       LOG.debug("scannerClose: id=" + id);
593       ResultScanner scanner = getScanner(id);
594       if (scanner == null) {
595         throw new IllegalArgument("scanner ID is invalid");
596       }
597       scanner.close();
598       removeScanner(id);
599     }
600 
601     public List<TRowResult> scannerGetList(int id,int nbRows) throws IllegalArgument, IOError {
602         LOG.debug("scannerGetList: id=" + id);
603         ResultScanner scanner = getScanner(id);
604         if (null == scanner) {
605             throw new IllegalArgument("scanner ID is invalid");
606         }
607 
608         Result [] results = null;
609         try {
610             results = scanner.next(nbRows);
611             if (null == results) {
612                 return new ArrayList<TRowResult>();
613             }
614         } catch (IOException e) {
615             throw new IOError(e.getMessage());
616         }
617         return ThriftUtilities.rowResultFromHBase(results);
618     }
619     public List<TRowResult> scannerGet(int id) throws IllegalArgument, IOError {
620         return scannerGetList(id,1);
621     }
622     public int scannerOpen(byte[] tableName, byte[] startRow,
623             List<byte[]> columns) throws IOError {
624         try {
625           HTable table = getTable(tableName);
626           Scan scan = new Scan(startRow);
627           if(columns != null && columns.size() != 0) {
628             for(byte [] column : columns) {
629               byte [][] famQf = KeyValue.parseColumn(column);
630               if(famQf.length == 1) {
631                 scan.addFamily(famQf[0]);
632               } else {
633                 scan.addColumn(famQf[0], famQf[1]);
634               }
635             }
636           }
637           return addScanner(table.getScanner(scan));
638         } catch (IOException e) {
639           throw new IOError(e.getMessage());
640         }
641     }
642 
643     public int scannerOpenWithStop(byte[] tableName, byte[] startRow,
644         byte[] stopRow, List<byte[]> columns) throws IOError, TException {
645       try {
646         HTable table = getTable(tableName);
647         Scan scan = new Scan(startRow, stopRow);
648         if(columns != null && columns.size() != 0) {
649           for(byte [] column : columns) {
650             byte [][] famQf = KeyValue.parseColumn(column);
651             if(famQf.length == 1) {
652               scan.addFamily(famQf[0]);
653             } else {
654               scan.addColumn(famQf[0], famQf[1]);
655             }
656           }
657         }
658         return addScanner(table.getScanner(scan));
659       } catch (IOException e) {
660         throw new IOError(e.getMessage());
661       }
662     }
663 
664     @Override
665     public int scannerOpenWithPrefix(byte[] tableName, byte[] startAndPrefix, List<byte[]> columns) throws IOError, TException {
666       try {
667         HTable table = getTable(tableName);
668         Scan scan = new Scan(startAndPrefix);
669         Filter f = new WhileMatchFilter(
670             new PrefixFilter(startAndPrefix));
671         scan.setFilter(f);
672         if(columns != null && columns.size() != 0) {
673           for(byte [] column : columns) {
674             byte [][] famQf = KeyValue.parseColumn(column);
675             if(famQf.length == 1) {
676               scan.addFamily(famQf[0]);
677             } else {
678               scan.addColumn(famQf[0], famQf[1]);
679             }
680           }
681         }
682         return addScanner(table.getScanner(scan));
683       } catch (IOException e) {
684         throw new IOError(e.getMessage());
685       }
686     }
687 
688     public int scannerOpenTs(byte[] tableName, byte[] startRow,
689         List<byte[]> columns, long timestamp) throws IOError, TException {
690       try {
691         HTable table = getTable(tableName);
692         Scan scan = new Scan(startRow);
693         scan.setTimeRange(Long.MIN_VALUE, timestamp);
694         if(columns != null && columns.size() != 0) {
695           for(byte [] column : columns) {
696             byte [][] famQf = KeyValue.parseColumn(column);
697             if(famQf.length == 1) {
698               scan.addFamily(famQf[0]);
699             } else {
700               scan.addColumn(famQf[0], famQf[1]);
701             }
702           }
703         }
704         return addScanner(table.getScanner(scan));
705       } catch (IOException e) {
706         throw new IOError(e.getMessage());
707       }
708     }
709 
710     public int scannerOpenWithStopTs(byte[] tableName, byte[] startRow,
711         byte[] stopRow, List<byte[]> columns, long timestamp)
712         throws IOError, TException {
713       try {
714         HTable table = getTable(tableName);
715         Scan scan = new Scan(startRow, stopRow);
716         scan.setTimeRange(Long.MIN_VALUE, timestamp);
717         if(columns != null && columns.size() != 0) {
718           for(byte [] column : columns) {
719             byte [][] famQf = KeyValue.parseColumn(column);
720             if(famQf.length == 1) {
721               scan.addFamily(famQf[0]);
722             } else {
723               scan.addColumn(famQf[0], famQf[1]);
724             }
725           }
726         }
727         scan.setTimeRange(Long.MIN_VALUE, timestamp);
728         return addScanner(table.getScanner(scan));
729       } catch (IOException e) {
730         throw new IOError(e.getMessage());
731       }
732     }
733 
734     public Map<byte[], ColumnDescriptor> getColumnDescriptors(
735         byte[] tableName) throws IOError, TException {
736       try {
737         TreeMap<byte[], ColumnDescriptor> columns =
738           new TreeMap<byte[], ColumnDescriptor>(Bytes.BYTES_COMPARATOR);
739 
740         HTable table = getTable(tableName);
741         HTableDescriptor desc = table.getTableDescriptor();
742 
743         for (HColumnDescriptor e : desc.getFamilies()) {
744           ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e);
745           columns.put(col.name, col);
746         }
747         return columns;
748       } catch (IOException e) {
749         throw new IOError(e.getMessage());
750       }
751     }
752   }
753 
754   //
755   // Main program and support routines
756   //
757 
758   private static void printUsageAndExit(Options options, int exitCode) {
759     HelpFormatter formatter = new HelpFormatter();
760     formatter.printHelp("Thrift", null, options,
761             "To start the Thrift server run 'bin/hbase-daemon.sh start thrift'\n" +
762             "To shutdown the thrift server run 'bin/hbase-daemon.sh stop thrift' or" +
763             " send a kill signal to the thrift server pid",
764             true);
765       System.exit(exitCode);
766   }
767 
768   private static final String DEFAULT_LISTEN_PORT = "9090";
769 
770   /*
771    * Start up the Thrift server.
772    * @param args
773    */
774   static private void doMain(final String[] args) throws Exception {
775     Log LOG = LogFactory.getLog("ThriftServer");
776 
777     Options options = new Options();
778     options.addOption("b", "bind", true, "Address to bind the Thrift server to. Not supported by the Nonblocking and HsHa server [default: 0.0.0.0]");
779     options.addOption("p", "port", true, "Port to bind to [default: 9090]");
780     options.addOption("f", "framed", false, "Use framed transport");
781     options.addOption("c", "compact", false, "Use the compact protocol");
782     options.addOption("h", "help", false, "Print help information");
783 
784     OptionGroup servers = new OptionGroup();
785     servers.addOption(new Option("nonblocking", false, "Use the TNonblockingServer. This implies the framed transport."));
786     servers.addOption(new Option("hsha", false, "Use the THsHaServer. This implies the framed transport."));
787     servers.addOption(new Option("threadpool", false, "Use the TThreadPoolServer. This is the default."));
788     options.addOptionGroup(servers);
789 
790     CommandLineParser parser = new PosixParser();
791     CommandLine cmd = parser.parse(options, args);
792 
793     /**
794      * This is so complicated to please both bin/hbase and bin/hbase-daemon.
795      * hbase-daemon provides "start" and "stop" arguments
796      * hbase should print the help if no argument is provided
797      */
798     List<String> commandLine = Arrays.asList(args);
799     boolean stop = commandLine.contains("stop");
800     boolean start = commandLine.contains("start");
801     if (cmd.hasOption("help") || !start || stop) {
802       printUsageAndExit(options, 1);
803     }
804 
805     // Get port to bind to
806     int listenPort = 0;
807     try {
808       listenPort = Integer.parseInt(cmd.getOptionValue("port", DEFAULT_LISTEN_PORT));
809     } catch (NumberFormatException e) {
810       LOG.error("Could not parse the value provided for the port option", e);
811       printUsageAndExit(options, -1);
812     }
813 
814     // Construct correct ProtocolFactory
815     TProtocolFactory protocolFactory;
816     if (cmd.hasOption("compact")) {
817       LOG.debug("Using compact protocol");
818       protocolFactory = new TCompactProtocol.Factory();
819     } else {
820       LOG.debug("Using binary protocol");
821       protocolFactory = new TBinaryProtocol.Factory();
822     }
823 
824     HBaseHandler handler = new HBaseHandler();
825     Hbase.Processor processor = new Hbase.Processor(handler);
826 
827     TServer server;
828     if (cmd.hasOption("nonblocking") || cmd.hasOption("hsha")) {
829       if (cmd.hasOption("bind")) {
830         LOG.error("The Nonblocking and HsHa servers don't support IP address binding at the moment." +
831                 " See https://issues.apache.org/jira/browse/HBASE-2155 for details.");
832         printUsageAndExit(options, -1);
833       }
834 
835       TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(listenPort);
836       TFramedTransport.Factory transportFactory = new TFramedTransport.Factory();
837 
838       if (cmd.hasOption("nonblocking")) {
839         LOG.info("starting HBase Nonblocking Thrift server on " + Integer.toString(listenPort));
840         server = new TNonblockingServer(processor, serverTransport, transportFactory, protocolFactory);
841       } else {
842         LOG.info("starting HBase HsHA Thrift server on " + Integer.toString(listenPort));
843         server = new THsHaServer(processor, serverTransport, transportFactory, protocolFactory);
844       }
845     } else {
846       // Get IP address to bind to
847       InetAddress listenAddress = null;
848       if (cmd.hasOption("bind")) {
849         try {
850           listenAddress = InetAddress.getByName(cmd.getOptionValue("bind"));
851         } catch (UnknownHostException e) {
852           LOG.error("Could not bind to provided ip address", e);
853           printUsageAndExit(options, -1);
854         }
855       } else {
856         listenAddress = InetAddress.getLocalHost();
857       }
858       TServerTransport serverTransport = new TServerSocket(new InetSocketAddress(listenAddress, listenPort));
859 
860       // Construct correct TransportFactory
861       TTransportFactory transportFactory;
862       if (cmd.hasOption("framed")) {
863         transportFactory = new TFramedTransport.Factory();
864         LOG.debug("Using framed transport");
865       } else {
866         transportFactory = new TTransportFactory();
867       }
868 
869       LOG.info("starting HBase ThreadPool Thrift server on " + listenAddress + ":" + Integer.toString(listenPort));
870       server = new TThreadPoolServer(processor, serverTransport, transportFactory, protocolFactory);
871     }
872 
873     server.serve();
874   }
875 
876   /**
877    * @param args
878    * @throws Exception
879    */
880   public static void main(String [] args) throws Exception {
881     doMain(args);
882   }
883 }