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