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 static org.apache.hadoop.hbase.util.Bytes.getBytes;
22  
23  import java.io.IOException;
24  import java.net.InetAddress;
25  import java.net.InetSocketAddress;
26  import java.net.UnknownHostException;
27  import java.nio.ByteBuffer;
28  import java.security.PrivilegedAction;
29  import java.util.ArrayList;
30  import java.util.Arrays;
31  import java.util.Collections;
32  import java.util.HashMap;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.TreeMap;
36  import java.util.concurrent.BlockingQueue;
37  import java.util.concurrent.ExecutorService;
38  import java.util.concurrent.LinkedBlockingQueue;
39  import java.util.concurrent.ThreadPoolExecutor;
40  import java.util.concurrent.TimeUnit;
41  
42  import javax.security.auth.callback.Callback;
43  import javax.security.auth.callback.UnsupportedCallbackException;
44  import javax.security.sasl.AuthorizeCallback;
45  import javax.security.sasl.Sasl;
46  import javax.security.sasl.SaslServer;
47  
48  import org.apache.commons.cli.CommandLine;
49  import org.apache.commons.cli.Option;
50  import org.apache.commons.cli.OptionGroup;
51  import org.apache.commons.logging.Log;
52  import org.apache.commons.logging.LogFactory;
53  import org.apache.hadoop.hbase.classification.InterfaceAudience;
54  import org.apache.hadoop.conf.Configuration;
55  import org.apache.hadoop.hbase.HBaseConfiguration;
56  import org.apache.hadoop.hbase.HColumnDescriptor;
57  import org.apache.hadoop.hbase.HConstants;
58  import org.apache.hadoop.hbase.HRegionInfo;
59  import org.apache.hadoop.hbase.HTableDescriptor;
60  import org.apache.hadoop.hbase.KeyValue;
61  import org.apache.hadoop.hbase.ServerName;
62  import org.apache.hadoop.hbase.TableName;
63  import org.apache.hadoop.hbase.TableNotFoundException;
64  import org.apache.hadoop.hbase.client.Delete;
65  import org.apache.hadoop.hbase.client.Durability;
66  import org.apache.hadoop.hbase.client.Get;
67  import org.apache.hadoop.hbase.client.HBaseAdmin;
68  import org.apache.hadoop.hbase.client.HTable;
69  import org.apache.hadoop.hbase.client.HTableInterface;
70  import org.apache.hadoop.hbase.client.Increment;
71  import org.apache.hadoop.hbase.client.OperationWithAttributes;
72  import org.apache.hadoop.hbase.client.Put;
73  import org.apache.hadoop.hbase.client.Result;
74  import org.apache.hadoop.hbase.client.ResultScanner;
75  import org.apache.hadoop.hbase.client.Scan;
76  import org.apache.hadoop.hbase.filter.Filter;
77  import org.apache.hadoop.hbase.filter.ParseFilter;
78  import org.apache.hadoop.hbase.filter.PrefixFilter;
79  import org.apache.hadoop.hbase.filter.WhileMatchFilter;
80  import org.apache.hadoop.hbase.security.SecurityUtil;
81  import org.apache.hadoop.hbase.security.UserProvider;
82  import org.apache.hadoop.hbase.thrift.CallQueue.Call;
83  import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
84  import org.apache.hadoop.hbase.thrift.generated.BatchMutation;
85  import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
86  import org.apache.hadoop.hbase.thrift.generated.Hbase;
87  import org.apache.hadoop.hbase.thrift.generated.IOError;
88  import org.apache.hadoop.hbase.thrift.generated.IllegalArgument;
89  import org.apache.hadoop.hbase.thrift.generated.Mutation;
90  import org.apache.hadoop.hbase.thrift.generated.TCell;
91  import org.apache.hadoop.hbase.thrift.generated.TIncrement;
92  import org.apache.hadoop.hbase.thrift.generated.TRegionInfo;
93  import org.apache.hadoop.hbase.thrift.generated.TRowResult;
94  import org.apache.hadoop.hbase.thrift.generated.TScan;
95  import org.apache.hadoop.hbase.util.Bytes;
96  import org.apache.hadoop.hbase.util.ConnectionCache;
97  import org.apache.hadoop.hbase.util.DNS;
98  import org.apache.hadoop.hbase.util.Strings;
99  import org.apache.hadoop.security.SaslRpcServer.SaslGssCallbackHandler;
100 import org.apache.hadoop.security.UserGroupInformation;
101 import org.apache.thrift.TException;
102 import org.apache.thrift.TProcessor;
103 import org.apache.thrift.protocol.TBinaryProtocol;
104 import org.apache.thrift.protocol.TCompactProtocol;
105 import org.apache.thrift.protocol.TProtocol;
106 import org.apache.thrift.protocol.TProtocolFactory;
107 import org.apache.thrift.server.THsHaServer;
108 import org.apache.thrift.server.TNonblockingServer;
109 import org.apache.thrift.server.TServer;
110 import org.apache.thrift.server.TThreadedSelectorServer;
111 import org.apache.thrift.transport.TFramedTransport;
112 import org.apache.thrift.transport.TNonblockingServerSocket;
113 import org.apache.thrift.transport.TNonblockingServerTransport;
114 import org.apache.thrift.transport.TSaslServerTransport;
115 import org.apache.thrift.transport.TServerSocket;
116 import org.apache.thrift.transport.TServerTransport;
117 import org.apache.thrift.transport.TTransportFactory;
118 
119 import com.google.common.base.Joiner;
120 import com.google.common.base.Throwables;
121 import com.google.common.util.concurrent.ThreadFactoryBuilder;
122 
123 /**
124  * ThriftServerRunner - this class starts up a Thrift server which implements
125  * the Hbase API specified in the Hbase.thrift IDL file.
126  */
127 @InterfaceAudience.Private
128 @SuppressWarnings("deprecation")
129 public class ThriftServerRunner implements Runnable {
130 
131   private static final Log LOG = LogFactory.getLog(ThriftServerRunner.class);
132 
133   static final String SERVER_TYPE_CONF_KEY =
134       "hbase.regionserver.thrift.server.type";
135 
136   static final String BIND_CONF_KEY = "hbase.regionserver.thrift.ipaddress";
137   static final String COMPACT_CONF_KEY = "hbase.regionserver.thrift.compact";
138   static final String FRAMED_CONF_KEY = "hbase.regionserver.thrift.framed";
139   static final String MAX_FRAME_SIZE_CONF_KEY = "hbase.regionserver.thrift.framed.max_frame_size_in_mb";
140   static final String PORT_CONF_KEY = "hbase.regionserver.thrift.port";
141   static final String COALESCE_INC_KEY = "hbase.regionserver.thrift.coalesceIncrement";
142 
143   /**
144    * Amount of time in milliseconds before a server thread will timeout
145    * waiting for client to send data on a connected socket. Currently,
146    * applies only to TBoundedThreadPoolServer
147    */
148   public static final String THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY =
149     "hbase.thrift.server.socket.read.timeout";
150   public static final int THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT = 60000;
151 
152   /**
153    * Thrift quality of protection configuration key. Valid values can be:
154    * auth-conf: authentication, integrity and confidentiality checking
155    * auth-int: authentication and integrity checking
156    * auth: authentication only
157    *
158    * This is used to authenticate the callers and support impersonation.
159    * The thrift server and the HBase cluster must run in secure mode.
160    */
161   static final String THRIFT_QOP_KEY = "hbase.thrift.security.qop";
162   static final String BACKLOG_CONF_KEY = "hbase.regionserver.thrift.backlog";
163 
164   private static final String DEFAULT_BIND_ADDR = "0.0.0.0";
165   public static final int DEFAULT_LISTEN_PORT = 9090;
166   private final int listenPort;
167 
168   private Configuration conf;
169   volatile TServer tserver;
170   private final Hbase.Iface handler;
171   private final ThriftMetrics metrics;
172   private final HBaseHandler hbaseHandler;
173   private final UserGroupInformation realUser;
174 
175   private final String qop;
176   private String host;
177 
178   /** An enum of server implementation selections */
179   enum ImplType {
180     HS_HA("hsha", true, THsHaServer.class, true),
181     NONBLOCKING("nonblocking", true, TNonblockingServer.class, true),
182     THREAD_POOL("threadpool", false, TBoundedThreadPoolServer.class, true),
183     THREADED_SELECTOR(
184         "threadedselector", true, TThreadedSelectorServer.class, true);
185 
186     public static final ImplType DEFAULT = THREAD_POOL;
187 
188     final String option;
189     final boolean isAlwaysFramed;
190     final Class<? extends TServer> serverClass;
191     final boolean canSpecifyBindIP;
192 
193     ImplType(String option, boolean isAlwaysFramed,
194         Class<? extends TServer> serverClass, boolean canSpecifyBindIP) {
195       this.option = option;
196       this.isAlwaysFramed = isAlwaysFramed;
197       this.serverClass = serverClass;
198       this.canSpecifyBindIP = canSpecifyBindIP;
199     }
200 
201     /**
202      * @return <code>-option</code> so we can get the list of options from
203      *         {@link #values()}
204      */
205     @Override
206     public String toString() {
207       return "-" + option;
208     }
209 
210     String getDescription() {
211       StringBuilder sb = new StringBuilder("Use the " +
212           serverClass.getSimpleName());
213       if (isAlwaysFramed) {
214         sb.append(" This implies the framed transport.");
215       }
216       if (this == DEFAULT) {
217         sb.append("This is the default.");
218       }
219       return sb.toString();
220     }
221 
222     static OptionGroup createOptionGroup() {
223       OptionGroup group = new OptionGroup();
224       for (ImplType t : values()) {
225         group.addOption(new Option(t.option, t.getDescription()));
226       }
227       return group;
228     }
229 
230     static ImplType getServerImpl(Configuration conf) {
231       String confType = conf.get(SERVER_TYPE_CONF_KEY, THREAD_POOL.option);
232       for (ImplType t : values()) {
233         if (confType.equals(t.option)) {
234           return t;
235         }
236       }
237       throw new AssertionError("Unknown server ImplType.option:" + confType);
238     }
239 
240     static void setServerImpl(CommandLine cmd, Configuration conf) {
241       ImplType chosenType = null;
242       int numChosen = 0;
243       for (ImplType t : values()) {
244         if (cmd.hasOption(t.option)) {
245           chosenType = t;
246           ++numChosen;
247         }
248       }
249       if (numChosen < 1) {
250         LOG.info("Using default thrift server type");
251         chosenType = DEFAULT;
252       } else if (numChosen > 1) {
253         throw new AssertionError("Exactly one option out of " +
254           Arrays.toString(values()) + " has to be specified");
255       }
256       LOG.info("Using thrift server type " + chosenType.option);
257       conf.set(SERVER_TYPE_CONF_KEY, chosenType.option);
258     }
259 
260     public String simpleClassName() {
261       return serverClass.getSimpleName();
262     }
263 
264     public static List<String> serversThatCannotSpecifyBindIP() {
265       List<String> l = new ArrayList<String>();
266       for (ImplType t : values()) {
267         if (!t.canSpecifyBindIP) {
268           l.add(t.simpleClassName());
269         }
270       }
271       return l;
272     }
273 
274   }
275 
276   public ThriftServerRunner(Configuration conf) throws IOException {
277     UserProvider userProvider = UserProvider.instantiate(conf);
278     // login the server principal (if using secure Hadoop)
279     boolean securityEnabled = userProvider.isHadoopSecurityEnabled()
280       && userProvider.isHBaseSecurityEnabled();
281     if (securityEnabled) {
282       host = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
283         conf.get("hbase.thrift.dns.interface", "default"),
284         conf.get("hbase.thrift.dns.nameserver", "default")));
285       userProvider.login("hbase.thrift.keytab.file",
286         "hbase.thrift.kerberos.principal", host);
287     }
288     this.conf = HBaseConfiguration.create(conf);
289     this.listenPort = conf.getInt(PORT_CONF_KEY, DEFAULT_LISTEN_PORT);
290     this.metrics = new ThriftMetrics(conf, ThriftMetrics.ThriftServerType.ONE);
291     this.hbaseHandler = new HBaseHandler(conf, userProvider);
292     this.hbaseHandler.initMetrics(metrics);
293     this.handler = HbaseHandlerMetricsProxy.newInstance(
294       hbaseHandler, metrics, conf);
295     this.realUser = userProvider.getCurrent().getUGI();
296     qop = conf.get(THRIFT_QOP_KEY);
297     if (qop != null) {
298       if (!qop.equals("auth") && !qop.equals("auth-int")
299           && !qop.equals("auth-conf")) {
300         throw new IOException("Invalid " + THRIFT_QOP_KEY + ": " + qop
301           + ", it must be 'auth', 'auth-int', or 'auth-conf'");
302       }
303       if (!securityEnabled) {
304         throw new IOException("Thrift server must"
305           + " run in secure mode to support authentication");
306       }
307     }
308   }
309 
310   /*
311    * Runs the Thrift server
312    */
313   @Override
314   public void run() {
315     realUser.doAs(
316       new PrivilegedAction<Object>() {
317         @Override
318         public Object run() {
319           try {
320             setupServer();
321             tserver.serve();
322           } catch (Exception e) {
323             LOG.fatal("Cannot run ThriftServer", e);
324             // Crash the process if the ThriftServer is not running
325             System.exit(-1);
326           }
327           return null;
328         }
329       });
330   }
331 
332   public void shutdown() {
333     if (tserver != null) {
334       tserver.stop();
335       tserver = null;
336     }
337   }
338 
339   /**
340    * Setting up the thrift TServer
341    */
342   private void setupServer() throws Exception {
343     // Construct correct ProtocolFactory
344     TProtocolFactory protocolFactory;
345     if (conf.getBoolean(COMPACT_CONF_KEY, false)) {
346       LOG.debug("Using compact protocol");
347       protocolFactory = new TCompactProtocol.Factory();
348     } else {
349       LOG.debug("Using binary protocol");
350       protocolFactory = new TBinaryProtocol.Factory();
351     }
352 
353     final TProcessor p = new Hbase.Processor<Hbase.Iface>(handler);
354     ImplType implType = ImplType.getServerImpl(conf);
355     TProcessor processor = p;
356 
357     // Construct correct TransportFactory
358     TTransportFactory transportFactory;
359     if (conf.getBoolean(FRAMED_CONF_KEY, false) || implType.isAlwaysFramed) {
360       if (qop != null) {
361         throw new RuntimeException("Thrift server authentication"
362           + " doesn't work with framed transport yet");
363       }
364       transportFactory = new TFramedTransport.Factory(
365           conf.getInt(MAX_FRAME_SIZE_CONF_KEY, 2)  * 1024 * 1024);
366       LOG.debug("Using framed transport");
367     } else if (qop == null) {
368       transportFactory = new TTransportFactory();
369     } else {
370       // Extract the name from the principal
371       String name = SecurityUtil.getUserFromPrincipal(
372         conf.get("hbase.thrift.kerberos.principal"));
373       Map<String, String> saslProperties = new HashMap<String, String>();
374       saslProperties.put(Sasl.QOP, qop);
375       TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory();
376       saslFactory.addServerDefinition("GSSAPI", name, host, saslProperties,
377         new SaslGssCallbackHandler() {
378           @Override
379           public void handle(Callback[] callbacks)
380               throws UnsupportedCallbackException {
381             AuthorizeCallback ac = null;
382             for (Callback callback : callbacks) {
383               if (callback instanceof AuthorizeCallback) {
384                 ac = (AuthorizeCallback) callback;
385               } else {
386                 throw new UnsupportedCallbackException(callback,
387                     "Unrecognized SASL GSSAPI Callback");
388               }
389             }
390             if (ac != null) {
391               String authid = ac.getAuthenticationID();
392               String authzid = ac.getAuthorizationID();
393               if (!authid.equals(authzid)) {
394                 ac.setAuthorized(false);
395               } else {
396                 ac.setAuthorized(true);
397                 String userName = SecurityUtil.getUserFromPrincipal(authzid);
398                 LOG.info("Effective user: " + userName);
399                 ac.setAuthorizedID(userName);
400               }
401             }
402           }
403         });
404       transportFactory = saslFactory;
405 
406       // Create a processor wrapper, to get the caller
407       processor = new TProcessor() {
408         @Override
409         public boolean process(TProtocol inProt,
410             TProtocol outProt) throws TException {
411           TSaslServerTransport saslServerTransport =
412             (TSaslServerTransport)inProt.getTransport();
413           SaslServer saslServer = saslServerTransport.getSaslServer();
414           String principal = saslServer.getAuthorizationID();
415           hbaseHandler.setEffectiveUser(principal);
416           return p.process(inProt, outProt);
417         }
418       };
419     }
420 
421     if (conf.get(BIND_CONF_KEY) != null && !implType.canSpecifyBindIP) {
422       LOG.error("Server types " + Joiner.on(", ").join(
423           ImplType.serversThatCannotSpecifyBindIP()) + " don't support IP " +
424           "address binding at the moment. See " +
425           "https://issues.apache.org/jira/browse/HBASE-2155 for details.");
426       throw new RuntimeException(
427           "-" + BIND_CONF_KEY + " not supported with " + implType);
428     }
429 
430     if (implType == ImplType.HS_HA || implType == ImplType.NONBLOCKING ||
431         implType == ImplType.THREADED_SELECTOR) {
432       InetAddress listenAddress = getBindAddress(conf);
433       TNonblockingServerTransport serverTransport = new TNonblockingServerSocket(
434           new InetSocketAddress(listenAddress, listenPort));
435 
436       if (implType == ImplType.NONBLOCKING) {
437         TNonblockingServer.Args serverArgs =
438             new TNonblockingServer.Args(serverTransport);
439         serverArgs.processor(processor)
440                   .transportFactory(transportFactory)
441                   .protocolFactory(protocolFactory);
442         tserver = new TNonblockingServer(serverArgs);
443       } else if (implType == ImplType.HS_HA) {
444         THsHaServer.Args serverArgs = new THsHaServer.Args(serverTransport);
445         CallQueue callQueue =
446             new CallQueue(new LinkedBlockingQueue<Call>(), metrics);
447         ExecutorService executorService = createExecutor(
448             callQueue, serverArgs.getWorkerThreads());
449         serverArgs.executorService(executorService)
450                   .processor(processor)
451                   .transportFactory(transportFactory)
452                   .protocolFactory(protocolFactory);
453         tserver = new THsHaServer(serverArgs);
454       } else { // THREADED_SELECTOR
455         TThreadedSelectorServer.Args serverArgs =
456             new HThreadedSelectorServerArgs(serverTransport, conf);
457         CallQueue callQueue =
458             new CallQueue(new LinkedBlockingQueue<Call>(), metrics);
459         ExecutorService executorService = createExecutor(
460             callQueue, serverArgs.getWorkerThreads());
461         serverArgs.executorService(executorService)
462                   .processor(processor)
463                   .transportFactory(transportFactory)
464                   .protocolFactory(protocolFactory);
465         tserver = new TThreadedSelectorServer(serverArgs);
466       }
467       LOG.info("starting HBase " + implType.simpleClassName() +
468           " server on " + Integer.toString(listenPort));
469     } else if (implType == ImplType.THREAD_POOL) {
470       // Thread pool server. Get the IP address to bind to.
471       InetAddress listenAddress = getBindAddress(conf);
472       int readTimeout = conf.getInt(THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY,
473           THRIFT_SERVER_SOCKET_READ_TIMEOUT_DEFAULT);
474       TServerTransport serverTransport =
475         new TServerSocket(new InetSocketAddress(listenAddress, listenPort), readTimeout);
476 
477       TBoundedThreadPoolServer.Args serverArgs =
478           new TBoundedThreadPoolServer.Args(serverTransport, conf);
479       serverArgs.processor(processor)
480                 .transportFactory(transportFactory)
481                 .protocolFactory(protocolFactory);
482       LOG.info("starting " + ImplType.THREAD_POOL.simpleClassName() + " on "
483           + listenAddress + ":" + Integer.toString(listenPort)
484           + " with readTimeout " + readTimeout + "ms; " + serverArgs);
485       TBoundedThreadPoolServer tserver =
486           new TBoundedThreadPoolServer(serverArgs, metrics);
487       this.tserver = tserver;
488     } else {
489       throw new AssertionError("Unsupported Thrift server implementation: " +
490           implType.simpleClassName());
491     }
492 
493     // A sanity check that we instantiated the right type of server.
494     if (tserver.getClass() != implType.serverClass) {
495       throw new AssertionError("Expected to create Thrift server class " +
496           implType.serverClass.getName() + " but got " +
497           tserver.getClass().getName());
498     }
499 
500 
501 
502     registerFilters(conf);
503   }
504 
505   ExecutorService createExecutor(BlockingQueue<Runnable> callQueue,
506                                  int workerThreads) {
507     ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
508     tfb.setDaemon(true);
509     tfb.setNameFormat("thrift-worker-%d");
510     return new ThreadPoolExecutor(workerThreads, workerThreads,
511             Long.MAX_VALUE, TimeUnit.SECONDS, callQueue, tfb.build());
512   }
513 
514   private InetAddress getBindAddress(Configuration conf)
515       throws UnknownHostException {
516     String bindAddressStr = conf.get(BIND_CONF_KEY, DEFAULT_BIND_ADDR);
517     return InetAddress.getByName(bindAddressStr);
518   }
519 
520   protected static class ResultScannerWrapper {
521 
522     private final ResultScanner scanner;
523     private final boolean sortColumns;
524     public ResultScannerWrapper(ResultScanner resultScanner,
525                                 boolean sortResultColumns) {
526       scanner = resultScanner;
527       sortColumns = sortResultColumns;
528    }
529 
530     public ResultScanner getScanner() {
531       return scanner;
532     }
533 
534     public boolean isColumnSorted() {
535       return sortColumns;
536     }
537   }
538 
539   /**
540    * The HBaseHandler is a glue object that connects Thrift RPC calls to the
541    * HBase client API primarily defined in the HBaseAdmin and HTable objects.
542    */
543   public static class HBaseHandler implements Hbase.Iface {
544     protected Configuration conf;
545     protected final Log LOG = LogFactory.getLog(this.getClass().getName());
546 
547     // nextScannerId and scannerMap are used to manage scanner state
548     protected int nextScannerId = 0;
549     protected HashMap<Integer, ResultScannerWrapper> scannerMap = null;
550     private ThriftMetrics metrics = null;
551 
552     private final ConnectionCache connectionCache;
553     IncrementCoalescer coalescer = null;
554 
555     static final String CLEANUP_INTERVAL = "hbase.thrift.connection.cleanup-interval";
556     static final String MAX_IDLETIME = "hbase.thrift.connection.max-idletime";
557 
558     /**
559      * Returns a list of all the column families for a given htable.
560      *
561      * @param table
562      * @throws IOException
563      */
564     byte[][] getAllColumns(HTable table) throws IOException {
565       HColumnDescriptor[] cds = table.getTableDescriptor().getColumnFamilies();
566       byte[][] columns = new byte[cds.length][];
567       for (int i = 0; i < cds.length; i++) {
568         columns[i] = Bytes.add(cds[i].getName(),
569             KeyValue.COLUMN_FAMILY_DELIM_ARRAY);
570       }
571       return columns;
572     }
573 
574     /**
575      * Creates and returns an HTable instance from a given table name.
576      *
577      * @param tableName
578      *          name of table
579      * @return HTable object
580      * @throws IOException
581      */
582     public HTableInterface getTable(final byte[] tableName) throws
583         IOException {
584       String table = Bytes.toString(tableName);
585       return connectionCache.getTable(table);
586     }
587 
588     public HTableInterface getTable(final ByteBuffer tableName) throws IOException {
589       return getTable(getBytes(tableName));
590     }
591 
592     /**
593      * Assigns a unique ID to the scanner and adds the mapping to an internal
594      * hash-map.
595      *
596      * @param scanner
597      * @return integer scanner id
598      */
599     protected synchronized int addScanner(ResultScanner scanner,boolean sortColumns) {
600       int id = nextScannerId++;
601       ResultScannerWrapper resultScannerWrapper = new ResultScannerWrapper(scanner, sortColumns);
602       scannerMap.put(id, resultScannerWrapper);
603       return id;
604     }
605 
606     /**
607      * Returns the scanner associated with the specified ID.
608      *
609      * @param id
610      * @return a Scanner, or null if ID was invalid.
611      */
612     protected synchronized ResultScannerWrapper getScanner(int id) {
613       return scannerMap.get(id);
614     }
615 
616     /**
617      * Removes the scanner associated with the specified ID from the internal
618      * id->scanner hash-map.
619      *
620      * @param id
621      * @return a Scanner, or null if ID was invalid.
622      */
623     protected synchronized ResultScannerWrapper removeScanner(int id) {
624       return scannerMap.remove(id);
625     }
626 
627     protected HBaseHandler(final Configuration c,
628         final UserProvider userProvider) throws IOException {
629       this.conf = c;
630       scannerMap = new HashMap<Integer, ResultScannerWrapper>();
631       this.coalescer = new IncrementCoalescer(this);
632 
633       int cleanInterval = conf.getInt(CLEANUP_INTERVAL, 10 * 1000);
634       int maxIdleTime = conf.getInt(MAX_IDLETIME, 10 * 60 * 1000);
635       connectionCache = new ConnectionCache(
636         conf, userProvider, cleanInterval, maxIdleTime);
637     }
638 
639     /**
640      * Obtain HBaseAdmin. Creates the instance if it is not already created.
641      */
642     private HBaseAdmin getHBaseAdmin() throws IOException {
643       return connectionCache.getAdmin();
644     }
645 
646     void setEffectiveUser(String effectiveUser) {
647       connectionCache.setEffectiveUser(effectiveUser);
648     }
649 
650     @Override
651     public void enableTable(ByteBuffer tableName) throws IOError {
652       try{
653         getHBaseAdmin().enableTable(getBytes(tableName));
654       } catch (IOException e) {
655         LOG.warn(e.getMessage(), e);
656         throw new IOError(e.getMessage());
657       }
658     }
659 
660     @Override
661     public void disableTable(ByteBuffer tableName) throws IOError{
662       try{
663         getHBaseAdmin().disableTable(getBytes(tableName));
664       } catch (IOException e) {
665         LOG.warn(e.getMessage(), e);
666         throw new IOError(e.getMessage());
667       }
668     }
669 
670     @Override
671     public boolean isTableEnabled(ByteBuffer tableName) throws IOError {
672       try {
673         return HTable.isTableEnabled(this.conf, getBytes(tableName));
674       } catch (IOException e) {
675         LOG.warn(e.getMessage(), e);
676         throw new IOError(e.getMessage());
677       }
678     }
679 
680     @Override
681     public void compact(ByteBuffer tableNameOrRegionName) throws IOError {
682       try{
683         getHBaseAdmin().compact(getBytes(tableNameOrRegionName));
684       } catch (InterruptedException e) {
685         throw new IOError(e.getMessage());
686       } catch (IOException e) {
687         LOG.warn(e.getMessage(), e);
688         throw new IOError(e.getMessage());
689       }
690     }
691 
692     @Override
693     public void majorCompact(ByteBuffer tableNameOrRegionName) throws IOError {
694       try{
695         getHBaseAdmin().majorCompact(getBytes(tableNameOrRegionName));
696       } catch (InterruptedException e) {
697         LOG.warn(e.getMessage(), e);
698         throw new IOError(e.getMessage());
699       } catch (IOException e) {
700         LOG.warn(e.getMessage(), e);
701         throw new IOError(e.getMessage());
702       }
703     }
704 
705     @Override
706     public List<ByteBuffer> getTableNames() throws IOError {
707       try {
708         TableName[] tableNames = this.getHBaseAdmin().listTableNames();
709         ArrayList<ByteBuffer> list = new ArrayList<ByteBuffer>(tableNames.length);
710         for (int i = 0; i < tableNames.length; i++) {
711           list.add(ByteBuffer.wrap(tableNames[i].getName()));
712         }
713         return list;
714       } catch (IOException e) {
715         LOG.warn(e.getMessage(), e);
716         throw new IOError(e.getMessage());
717       }
718     }
719 
720     /**
721      * @return the list of regions in the given table, or an empty list if the table does not exist
722      */
723     @Override
724     public List<TRegionInfo> getTableRegions(ByteBuffer tableName)
725     throws IOError {
726       try {
727         HTableInterface table;
728         try {
729           table = getTable(tableName);
730         } catch (TableNotFoundException ex) {
731           return new ArrayList<TRegionInfo>();
732         }
733         Map<HRegionInfo, ServerName> regionLocations = ((HTable)table).getRegionLocations();
734         List<TRegionInfo> results = new ArrayList<TRegionInfo>();
735         for (Map.Entry<HRegionInfo, ServerName> entry :
736             regionLocations.entrySet()) {
737           HRegionInfo info = entry.getKey();
738           ServerName serverName = entry.getValue();
739           TRegionInfo region = new TRegionInfo();
740           region.serverName = ByteBuffer.wrap(
741               Bytes.toBytes(serverName.getHostname()));
742           region.port = serverName.getPort();
743           region.startKey = ByteBuffer.wrap(info.getStartKey());
744           region.endKey = ByteBuffer.wrap(info.getEndKey());
745           region.id = info.getRegionId();
746           region.name = ByteBuffer.wrap(info.getRegionName());
747           region.version = info.getVersion();
748           results.add(region);
749         }
750         return results;
751       } catch (TableNotFoundException e) {
752         // Return empty list for non-existing table
753         return Collections.emptyList();
754       } catch (IOException e){
755         LOG.warn(e.getMessage(), e);
756         throw new IOError(e.getMessage());
757       }
758     }
759 
760     @Deprecated
761     @Override
762     public List<TCell> get(
763         ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
764         Map<ByteBuffer, ByteBuffer> attributes)
765         throws IOError {
766       byte [][] famAndQf = KeyValue.parseColumn(getBytes(column));
767       if (famAndQf.length == 1) {
768         return get(tableName, row, famAndQf[0], null, attributes);
769       }
770       if (famAndQf.length == 2) {
771         return get(tableName, row, famAndQf[0], famAndQf[1], attributes);
772       }
773       throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
774     }
775 
776     /**
777      * Note: this internal interface is slightly different from public APIs in regard to handling
778      * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
779      * we respect qual == null as a request for the entire column family. The caller (
780      * {@link #get(ByteBuffer, ByteBuffer, ByteBuffer, Map)}) interface IS consistent in that the
781      * column is parse like normal.
782      */
783     protected List<TCell> get(ByteBuffer tableName,
784                               ByteBuffer row,
785                               byte[] family,
786                               byte[] qualifier,
787                               Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
788       HTableInterface table = null;
789       try {
790         table = getTable(tableName);
791         Get get = new Get(getBytes(row));
792         addAttributes(get, attributes);
793         if (qualifier == null) {
794           get.addFamily(family);
795         } else {
796           get.addColumn(family, qualifier);
797         }
798         Result result = table.get(get);
799         return ThriftUtilities.cellFromHBase(result.rawCells());
800       } catch (IOException e) {
801         LOG.warn(e.getMessage(), e);
802         throw new IOError(Throwables.getStackTraceAsString(e));
803       } finally {
804         closeTable(table);
805       }
806     }
807 
808     @Deprecated
809     @Override
810     public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
811         int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
812       byte [][] famAndQf = KeyValue.parseColumn(getBytes(column));
813       if(famAndQf.length == 1) {
814         return getVer(tableName, row, famAndQf[0], null, numVersions, attributes);
815       }
816       if (famAndQf.length == 2) {
817         return getVer(tableName, row, famAndQf[0], famAndQf[1], numVersions, attributes);
818       }
819       throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
820 
821     }
822 
823     /**
824      * Note: this public interface is slightly different from public Java APIs in regard to
825      * handling of the qualifier. Here we differ from the public Java API in that null != byte[0].
826      * Rather, we respect qual == null as a request for the entire column family. If you want to
827      * access the entire column family, use
828      * {@link #getVer(ByteBuffer, ByteBuffer, ByteBuffer, int, Map)} with a {@code column} value
829      * that lacks a {@code ':'}.
830      */
831     public List<TCell> getVer(ByteBuffer tableName, ByteBuffer row, byte[] family,
832         byte[] qualifier, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
833       HTableInterface table = null;
834       try {
835         table = getTable(tableName);
836         Get get = new Get(getBytes(row));
837         addAttributes(get, attributes);
838         if (null == qualifier) {
839           get.addFamily(family);
840         } else {
841           get.addColumn(family, qualifier);
842         }
843         get.setMaxVersions(numVersions);
844         Result result = table.get(get);
845         return ThriftUtilities.cellFromHBase(result.rawCells());
846       } catch (IOException e) {
847         LOG.warn(e.getMessage(), e);
848         throw new IOError(Throwables.getStackTraceAsString(e));
849       } finally{
850         closeTable(table);
851       }
852     }
853 
854     @Deprecated
855     @Override
856     public List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
857         long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
858       byte [][] famAndQf = KeyValue.parseColumn(getBytes(column));
859       if (famAndQf.length == 1) {
860         return getVerTs(tableName, row, famAndQf[0], null, timestamp, numVersions, attributes);
861       }
862       if (famAndQf.length == 2) {
863         return getVerTs(tableName, row, famAndQf[0], famAndQf[1], timestamp, numVersions,
864           attributes);
865       }
866       throw new IllegalArgumentException("Invalid familyAndQualifier provided.");
867     }
868 
869     /**
870      * Note: this internal interface is slightly different from public APIs in regard to handling
871      * of the qualifier. Here we differ from the public Java API in that null != byte[0]. Rather,
872      * we respect qual == null as a request for the entire column family. The caller (
873      * {@link #getVerTs(ByteBuffer, ByteBuffer, ByteBuffer, long, int, Map)}) interface IS
874      * consistent in that the column is parse like normal.
875      */
876     protected List<TCell> getVerTs(ByteBuffer tableName, ByteBuffer row, byte[] family,
877         byte[] qualifier, long timestamp, int numVersions, Map<ByteBuffer, ByteBuffer> attributes)
878         throws IOError {
879       HTableInterface table = null;
880       try {
881         table = getTable(tableName);
882         Get get = new Get(getBytes(row));
883         addAttributes(get, attributes);
884         if (null == qualifier) {
885           get.addFamily(family);
886         } else {
887           get.addColumn(family, qualifier);
888         }
889         get.setTimeRange(0, timestamp);
890         get.setMaxVersions(numVersions);
891         Result result = table.get(get);
892         return ThriftUtilities.cellFromHBase(result.rawCells());
893       } catch (IOException e) {
894         LOG.warn(e.getMessage(), e);
895         throw new IOError(Throwables.getStackTraceAsString(e));
896       } finally{
897         closeTable(table);
898       }
899     }
900 
901     @Override
902     public List<TRowResult> getRow(ByteBuffer tableName, ByteBuffer row,
903         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
904       return getRowWithColumnsTs(tableName, row, null,
905                                  HConstants.LATEST_TIMESTAMP,
906                                  attributes);
907     }
908 
909     @Override
910     public List<TRowResult> getRowWithColumns(ByteBuffer tableName,
911                                               ByteBuffer row,
912         List<ByteBuffer> columns,
913         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
914       return getRowWithColumnsTs(tableName, row, columns,
915                                  HConstants.LATEST_TIMESTAMP,
916                                  attributes);
917     }
918 
919     @Override
920     public List<TRowResult> getRowTs(ByteBuffer tableName, ByteBuffer row,
921         long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
922       return getRowWithColumnsTs(tableName, row, null,
923                                  timestamp, attributes);
924     }
925 
926     @Override
927     public List<TRowResult> getRowWithColumnsTs(
928         ByteBuffer tableName, ByteBuffer row, List<ByteBuffer> columns,
929         long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
930       HTableInterface table = null;
931       try {
932         table = getTable(tableName);
933         if (columns == null) {
934           Get get = new Get(getBytes(row));
935           addAttributes(get, attributes);
936           get.setTimeRange(0, timestamp);
937           Result result = table.get(get);
938           return ThriftUtilities.rowResultFromHBase(result);
939         }
940         Get get = new Get(getBytes(row));
941         addAttributes(get, attributes);
942         for(ByteBuffer column : columns) {
943           byte [][] famAndQf = KeyValue.parseColumn(getBytes(column));
944           if (famAndQf.length == 1) {
945               get.addFamily(famAndQf[0]);
946           } else {
947               get.addColumn(famAndQf[0], famAndQf[1]);
948           }
949         }
950         get.setTimeRange(0, timestamp);
951         Result result = table.get(get);
952         return ThriftUtilities.rowResultFromHBase(result);
953       } catch (IOException e) {
954         LOG.warn(e.getMessage(), e);
955         throw new IOError(Throwables.getStackTraceAsString(e));
956       } finally{
957         closeTable(table);
958       }
959     }
960 
961     @Override
962     public List<TRowResult> getRows(ByteBuffer tableName,
963                                     List<ByteBuffer> rows,
964         Map<ByteBuffer, ByteBuffer> attributes)
965         throws IOError {
966       return getRowsWithColumnsTs(tableName, rows, null,
967                                   HConstants.LATEST_TIMESTAMP,
968                                   attributes);
969     }
970 
971     @Override
972     public List<TRowResult> getRowsWithColumns(ByteBuffer tableName,
973                                                List<ByteBuffer> rows,
974         List<ByteBuffer> columns,
975         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
976       return getRowsWithColumnsTs(tableName, rows, columns,
977                                   HConstants.LATEST_TIMESTAMP,
978                                   attributes);
979     }
980 
981     @Override
982     public List<TRowResult> getRowsTs(ByteBuffer tableName,
983                                       List<ByteBuffer> rows,
984         long timestamp,
985         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
986       return getRowsWithColumnsTs(tableName, rows, null,
987                                   timestamp, attributes);
988     }
989 
990     @Override
991     public List<TRowResult> getRowsWithColumnsTs(ByteBuffer tableName,
992                                                  List<ByteBuffer> rows,
993         List<ByteBuffer> columns, long timestamp,
994         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
995       HTableInterface table = null;
996       try {
997         List<Get> gets = new ArrayList<Get>(rows.size());
998         table = getTable(tableName);
999         if (metrics != null) {
1000           metrics.incNumRowKeysInBatchGet(rows.size());
1001         }
1002         for (ByteBuffer row : rows) {
1003           Get get = new Get(getBytes(row));
1004           addAttributes(get, attributes);
1005           if (columns != null) {
1006 
1007             for(ByteBuffer column : columns) {
1008               byte [][] famAndQf = KeyValue.parseColumn(getBytes(column));
1009               if (famAndQf.length == 1) {
1010                 get.addFamily(famAndQf[0]);
1011               } else {
1012                 get.addColumn(famAndQf[0], famAndQf[1]);
1013               }
1014             }
1015           }
1016           get.setTimeRange(0, timestamp);
1017           gets.add(get);
1018         }
1019         Result[] result = table.get(gets);
1020         return ThriftUtilities.rowResultFromHBase(result);
1021       } catch (IOException e) {
1022         LOG.warn(e.getMessage(), e);
1023         throw new IOError(Throwables.getStackTraceAsString(e));
1024       } finally{
1025         closeTable(table);
1026       }
1027     }
1028 
1029     @Override
1030     public void deleteAll(
1031         ByteBuffer tableName, ByteBuffer row, ByteBuffer column,
1032         Map<ByteBuffer, ByteBuffer> attributes)
1033         throws IOError {
1034       deleteAllTs(tableName, row, column, HConstants.LATEST_TIMESTAMP,
1035                   attributes);
1036     }
1037 
1038     @Override
1039     public void deleteAllTs(ByteBuffer tableName,
1040                             ByteBuffer row,
1041                             ByteBuffer column,
1042         long timestamp, Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
1043       HTableInterface table = null;
1044       try {
1045         table = getTable(tableName);
1046         Delete delete  = new Delete(getBytes(row));
1047         addAttributes(delete, attributes);
1048         byte [][] famAndQf = KeyValue.parseColumn(getBytes(column));
1049         if (famAndQf.length == 1) {
1050           delete.deleteFamily(famAndQf[0], timestamp);
1051         } else {
1052           delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
1053         }
1054         table.delete(delete);
1055 
1056       } catch (IOException e) {
1057         LOG.warn(e.getMessage(), e);
1058         throw new IOError(Throwables.getStackTraceAsString(e));
1059       } finally {
1060         closeTable(table);
1061       }
1062     }
1063 
1064     @Override
1065     public void deleteAllRow(
1066         ByteBuffer tableName, ByteBuffer row,
1067         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
1068       deleteAllRowTs(tableName, row, HConstants.LATEST_TIMESTAMP, attributes);
1069     }
1070 
1071     @Override
1072     public void deleteAllRowTs(
1073         ByteBuffer tableName, ByteBuffer row, long timestamp,
1074         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
1075       HTableInterface table = null;
1076       try {
1077         table = getTable(tableName);
1078         Delete delete  = new Delete(getBytes(row), timestamp);
1079         addAttributes(delete, attributes);
1080         table.delete(delete);
1081       } catch (IOException e) {
1082         LOG.warn(e.getMessage(), e);
1083         throw new IOError(Throwables.getStackTraceAsString(e));
1084       } finally {
1085         closeTable(table);
1086       }
1087     }
1088 
1089     @Override
1090     public void createTable(ByteBuffer in_tableName,
1091         List<ColumnDescriptor> columnFamilies) throws IOError,
1092         IllegalArgument, AlreadyExists {
1093       byte [] tableName = getBytes(in_tableName);
1094       try {
1095         if (getHBaseAdmin().tableExists(tableName)) {
1096           throw new AlreadyExists("table name already in use");
1097         }
1098         HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableName));
1099         for (ColumnDescriptor col : columnFamilies) {
1100           HColumnDescriptor colDesc = ThriftUtilities.colDescFromThrift(col);
1101           desc.addFamily(colDesc);
1102         }
1103         getHBaseAdmin().createTable(desc);
1104       } catch (IOException e) {
1105         LOG.warn(e.getMessage(), e);
1106         throw new IOError(e.getMessage());
1107       } catch (IllegalArgumentException e) {
1108         LOG.warn(e.getMessage(), e);
1109         throw new IllegalArgument(e.getMessage());
1110       }
1111     }
1112 
1113     @Override
1114     public void deleteTable(ByteBuffer in_tableName) throws IOError {
1115       byte [] tableName = getBytes(in_tableName);
1116       if (LOG.isDebugEnabled()) {
1117         LOG.debug("deleteTable: table=" + Bytes.toString(tableName));
1118       }
1119       try {
1120         if (!getHBaseAdmin().tableExists(tableName)) {
1121           throw new IOException("table does not exist");
1122         }
1123         getHBaseAdmin().deleteTable(tableName);
1124       } catch (IOException e) {
1125         LOG.warn(e.getMessage(), e);
1126         throw new IOError(e.getMessage());
1127       }
1128     }
1129 
1130     @Override
1131     public void mutateRow(ByteBuffer tableName, ByteBuffer row,
1132         List<Mutation> mutations, Map<ByteBuffer, ByteBuffer> attributes)
1133         throws IOError, IllegalArgument {
1134       mutateRowTs(tableName, row, mutations, HConstants.LATEST_TIMESTAMP,
1135                   attributes);
1136     }
1137 
1138     @Override
1139     public void mutateRowTs(ByteBuffer tableName, ByteBuffer row,
1140         List<Mutation> mutations, long timestamp,
1141         Map<ByteBuffer, ByteBuffer> attributes)
1142         throws IOError, IllegalArgument {
1143       HTableInterface table = null;
1144       try {
1145         table = getTable(tableName);
1146         Put put = new Put(getBytes(row), timestamp);
1147         addAttributes(put, attributes);
1148 
1149         Delete delete = new Delete(getBytes(row));
1150         addAttributes(delete, attributes);
1151         if (metrics != null) {
1152           metrics.incNumRowKeysInBatchMutate(mutations.size());
1153         }
1154 
1155         // I apologize for all this mess :)
1156         for (Mutation m : mutations) {
1157           byte[][] famAndQf = KeyValue.parseColumn(getBytes(m.column));
1158           if (m.isDelete) {
1159             if (famAndQf.length == 1) {
1160               delete.deleteFamily(famAndQf[0], timestamp);
1161             } else {
1162               delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
1163             }
1164             delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL
1165                 : Durability.SKIP_WAL);
1166           } else {
1167             if(famAndQf.length == 1) {
1168               LOG.warn("No column qualifier specified. Delete is the only mutation supported "
1169                   + "over the whole column family.");
1170             } else {
1171               put.addImmutable(famAndQf[0], famAndQf[1],
1172                   m.value != null ? getBytes(m.value)
1173                       : HConstants.EMPTY_BYTE_ARRAY);
1174             }
1175             put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
1176           }
1177         }
1178         if (!delete.isEmpty())
1179           table.delete(delete);
1180         if (!put.isEmpty())
1181           table.put(put);
1182       } catch (IOException e) {
1183         LOG.warn(e.getMessage(), e);
1184         throw new IOError(e.getMessage());
1185       } catch (IllegalArgumentException e) {
1186         LOG.warn(e.getMessage(), e);
1187         throw new IllegalArgument(Throwables.getStackTraceAsString(e));
1188       } finally{
1189         closeTable(table);
1190       }
1191     }
1192 
1193     @Override
1194     public void mutateRows(ByteBuffer tableName, List<BatchMutation> rowBatches,
1195         Map<ByteBuffer, ByteBuffer> attributes)
1196         throws IOError, IllegalArgument, TException {
1197       mutateRowsTs(tableName, rowBatches, HConstants.LATEST_TIMESTAMP, attributes);
1198     }
1199 
1200     @Override
1201     public void mutateRowsTs(
1202         ByteBuffer tableName, List<BatchMutation> rowBatches, long timestamp,
1203         Map<ByteBuffer, ByteBuffer> attributes)
1204         throws IOError, IllegalArgument, TException {
1205       List<Put> puts = new ArrayList<Put>();
1206       List<Delete> deletes = new ArrayList<Delete>();
1207 
1208       for (BatchMutation batch : rowBatches) {
1209         byte[] row = getBytes(batch.row);
1210         List<Mutation> mutations = batch.mutations;
1211         Delete delete = new Delete(row);
1212         addAttributes(delete, attributes);
1213         Put put = new Put(row, timestamp);
1214         addAttributes(put, attributes);
1215         for (Mutation m : mutations) {
1216           byte[][] famAndQf = KeyValue.parseColumn(getBytes(m.column));
1217           if (m.isDelete) {
1218             // no qualifier, family only.
1219             if (famAndQf.length == 1) {
1220               delete.deleteFamily(famAndQf[0], timestamp);
1221             } else {
1222               delete.deleteColumns(famAndQf[0], famAndQf[1], timestamp);
1223             }
1224             delete.setDurability(m.writeToWAL ? Durability.SYNC_WAL
1225                 : Durability.SKIP_WAL);
1226           } else {
1227             if (famAndQf.length == 1) {
1228               LOG.warn("No column qualifier specified. Delete is the only mutation supported "
1229                   + "over the whole column family.");
1230             }
1231             if (famAndQf.length == 2) {
1232               put.addImmutable(famAndQf[0], famAndQf[1],
1233                   m.value != null ? getBytes(m.value)
1234                       : HConstants.EMPTY_BYTE_ARRAY);
1235             } else {
1236               throw new IllegalArgumentException("Invalid famAndQf provided.");
1237             }
1238             put.setDurability(m.writeToWAL ? Durability.SYNC_WAL : Durability.SKIP_WAL);
1239           }
1240         }
1241         if (!delete.isEmpty())
1242           deletes.add(delete);
1243         if (!put.isEmpty())
1244           puts.add(put);
1245       }
1246       HTableInterface table = null;
1247       try {
1248         table = getTable(tableName);
1249         if (!puts.isEmpty())
1250           table.put(puts);
1251         if (!deletes.isEmpty())
1252           table.delete(deletes);
1253 
1254       } catch (IOException e) {
1255         LOG.warn(e.getMessage(), e);
1256         throw new IOError(e.getMessage());
1257       } catch (IllegalArgumentException e) {
1258         LOG.warn(e.getMessage(), e);
1259         throw new IllegalArgument(Throwables.getStackTraceAsString(e));
1260       } finally{
1261         closeTable(table);
1262       }
1263     }
1264 
1265     @Deprecated
1266     @Override
1267     public long atomicIncrement(
1268         ByteBuffer tableName, ByteBuffer row, ByteBuffer column, long amount)
1269             throws IOError, IllegalArgument, TException {
1270       byte [][] famAndQf = KeyValue.parseColumn(getBytes(column));
1271       if(famAndQf.length == 1) {
1272         return atomicIncrement(tableName, row, famAndQf[0], HConstants.EMPTY_BYTE_ARRAY, amount);
1273       }
1274       return atomicIncrement(tableName, row, famAndQf[0], famAndQf[1], amount);
1275     }
1276 
1277     protected long atomicIncrement(ByteBuffer tableName, ByteBuffer row,
1278         byte [] family, byte [] qualifier, long amount)
1279         throws IOError, IllegalArgument, TException {
1280       HTableInterface table = null;
1281       try {
1282         table = getTable(tableName);
1283         return table.incrementColumnValue(
1284             getBytes(row), family, qualifier, amount);
1285       } catch (IOException e) {
1286         LOG.warn(e.getMessage(), e);
1287         throw new IOError(Throwables.getStackTraceAsString(e));
1288       } finally {
1289         closeTable(table);
1290       }
1291     }
1292 
1293     @Override
1294     public void scannerClose(int id) throws IOError, IllegalArgument {
1295       LOG.debug("scannerClose: id=" + id);
1296       ResultScannerWrapper resultScannerWrapper = getScanner(id);
1297       if (resultScannerWrapper == null) {
1298         String message = "scanner ID is invalid";
1299         LOG.warn(message);
1300         throw new IllegalArgument("scanner ID is invalid");
1301       }
1302       resultScannerWrapper.getScanner().close();
1303       removeScanner(id);
1304     }
1305 
1306     @Override
1307     public List<TRowResult> scannerGetList(int id,int nbRows)
1308         throws IllegalArgument, IOError {
1309       LOG.debug("scannerGetList: id=" + id);
1310       ResultScannerWrapper resultScannerWrapper = getScanner(id);
1311       if (null == resultScannerWrapper) {
1312         String message = "scanner ID is invalid";
1313         LOG.warn(message);
1314         throw new IllegalArgument("scanner ID is invalid");
1315       }
1316 
1317       Result [] results = null;
1318       try {
1319         results = resultScannerWrapper.getScanner().next(nbRows);
1320         if (null == results) {
1321           return new ArrayList<TRowResult>();
1322         }
1323       } catch (IOException e) {
1324         LOG.warn(e.getMessage(), e);
1325         throw new IOError(e.getMessage());
1326       }
1327       return ThriftUtilities.rowResultFromHBase(results, resultScannerWrapper.isColumnSorted());
1328     }
1329 
1330     @Override
1331     public List<TRowResult> scannerGet(int id) throws IllegalArgument, IOError {
1332       return scannerGetList(id,1);
1333     }
1334 
1335     @Override
1336     public int scannerOpenWithScan(ByteBuffer tableName, TScan tScan,
1337         Map<ByteBuffer, ByteBuffer> attributes)
1338         throws IOError {
1339       HTableInterface table = null;
1340       try {
1341         table = getTable(tableName);
1342         Scan scan = new Scan();
1343         addAttributes(scan, attributes);
1344         if (tScan.isSetStartRow()) {
1345           scan.setStartRow(tScan.getStartRow());
1346         }
1347         if (tScan.isSetStopRow()) {
1348           scan.setStopRow(tScan.getStopRow());
1349         }
1350         if (tScan.isSetTimestamp()) {
1351           scan.setTimeRange(0, tScan.getTimestamp());
1352         }
1353         if (tScan.isSetCaching()) {
1354           scan.setCaching(tScan.getCaching());
1355         }
1356         if (tScan.isSetBatchSize()) {
1357           scan.setBatch(tScan.getBatchSize());
1358         }
1359         if (tScan.isSetColumns() && tScan.getColumns().size() != 0) {
1360           for(ByteBuffer column : tScan.getColumns()) {
1361             byte [][] famQf = KeyValue.parseColumn(getBytes(column));
1362             if(famQf.length == 1) {
1363               scan.addFamily(famQf[0]);
1364             } else {
1365               scan.addColumn(famQf[0], famQf[1]);
1366             }
1367           }
1368         }
1369         if (tScan.isSetFilterString()) {
1370           ParseFilter parseFilter = new ParseFilter();
1371           scan.setFilter(
1372               parseFilter.parseFilterString(tScan.getFilterString()));
1373         }
1374         if (tScan.isSetReversed()) {
1375           scan.setReversed(tScan.isReversed());
1376         }
1377         return addScanner(table.getScanner(scan), tScan.sortColumns);
1378       } catch (IOException e) {
1379         LOG.warn(e.getMessage(), e);
1380         throw new IOError(Throwables.getStackTraceAsString(e));
1381       } finally{
1382         closeTable(table);
1383       }
1384     }
1385 
1386     @Override
1387     public int scannerOpen(ByteBuffer tableName, ByteBuffer startRow,
1388         List<ByteBuffer> columns,
1389         Map<ByteBuffer, ByteBuffer> attributes) throws IOError {
1390       HTableInterface table = null;
1391       try {
1392         table = getTable(tableName);
1393         Scan scan = new Scan(getBytes(startRow));
1394         addAttributes(scan, attributes);
1395         if(columns != null && columns.size() != 0) {
1396           for(ByteBuffer column : columns) {
1397             byte [][] famQf = KeyValue.parseColumn(getBytes(column));
1398             if(famQf.length == 1) {
1399               scan.addFamily(famQf[0]);
1400             } else {
1401               scan.addColumn(famQf[0], famQf[1]);
1402             }
1403           }
1404         }
1405         return addScanner(table.getScanner(scan), false);
1406       } catch (IOException e) {
1407         LOG.warn(e.getMessage(), e);
1408         throw new IOError(Throwables.getStackTraceAsString(e));
1409       } finally{
1410         closeTable(table);
1411       }
1412     }
1413 
1414     @Override
1415     public int scannerOpenWithStop(ByteBuffer tableName, ByteBuffer startRow,
1416         ByteBuffer stopRow, List<ByteBuffer> columns,
1417         Map<ByteBuffer, ByteBuffer> attributes)
1418         throws IOError, TException {
1419       HTableInterface table = null;
1420       try {
1421         table = getTable(tableName);
1422         Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
1423         addAttributes(scan, attributes);
1424         if(columns != null && columns.size() != 0) {
1425           for(ByteBuffer column : columns) {
1426             byte [][] famQf = KeyValue.parseColumn(getBytes(column));
1427             if(famQf.length == 1) {
1428               scan.addFamily(famQf[0]);
1429             } else {
1430               scan.addColumn(famQf[0], famQf[1]);
1431             }
1432           }
1433         }
1434         return addScanner(table.getScanner(scan), false);
1435       } catch (IOException e) {
1436         LOG.warn(e.getMessage(), e);
1437         throw new IOError(Throwables.getStackTraceAsString(e));
1438       } finally{
1439         closeTable(table);
1440       }
1441     }
1442 
1443     @Override
1444     public int scannerOpenWithPrefix(ByteBuffer tableName,
1445                                      ByteBuffer startAndPrefix,
1446                                      List<ByteBuffer> columns,
1447         Map<ByteBuffer, ByteBuffer> attributes)
1448         throws IOError, TException {
1449       HTableInterface table = null;
1450       try {
1451         table = getTable(tableName);
1452         Scan scan = new Scan(getBytes(startAndPrefix));
1453         addAttributes(scan, attributes);
1454         Filter f = new WhileMatchFilter(
1455             new PrefixFilter(getBytes(startAndPrefix)));
1456         scan.setFilter(f);
1457         if (columns != null && columns.size() != 0) {
1458           for(ByteBuffer column : columns) {
1459             byte [][] famQf = KeyValue.parseColumn(getBytes(column));
1460             if(famQf.length == 1) {
1461               scan.addFamily(famQf[0]);
1462             } else {
1463               scan.addColumn(famQf[0], famQf[1]);
1464             }
1465           }
1466         }
1467         return addScanner(table.getScanner(scan), false);
1468       } catch (IOException e) {
1469         LOG.warn(e.getMessage(), e);
1470         throw new IOError(Throwables.getStackTraceAsString(e));
1471       } finally{
1472         closeTable(table);
1473       }
1474     }
1475 
1476     @Override
1477     public int scannerOpenTs(ByteBuffer tableName, ByteBuffer startRow,
1478         List<ByteBuffer> columns, long timestamp,
1479         Map<ByteBuffer, ByteBuffer> attributes) throws IOError, TException {
1480       HTableInterface table = null;
1481       try {
1482         table = getTable(tableName);
1483         Scan scan = new Scan(getBytes(startRow));
1484         addAttributes(scan, attributes);
1485         scan.setTimeRange(0, timestamp);
1486         if (columns != null && columns.size() != 0) {
1487           for (ByteBuffer column : columns) {
1488             byte [][] famQf = KeyValue.parseColumn(getBytes(column));
1489             if(famQf.length == 1) {
1490               scan.addFamily(famQf[0]);
1491             } else {
1492               scan.addColumn(famQf[0], famQf[1]);
1493             }
1494           }
1495         }
1496         return addScanner(table.getScanner(scan), false);
1497       } catch (IOException e) {
1498         LOG.warn(e.getMessage(), e);
1499         throw new IOError(Throwables.getStackTraceAsString(e));
1500       } finally{
1501         closeTable(table);
1502       }
1503     }
1504 
1505     @Override
1506     public int scannerOpenWithStopTs(ByteBuffer tableName, ByteBuffer startRow,
1507         ByteBuffer stopRow, List<ByteBuffer> columns, long timestamp,
1508         Map<ByteBuffer, ByteBuffer> attributes)
1509         throws IOError, TException {
1510       HTableInterface table = null;
1511       try {
1512         table = getTable(tableName);
1513         Scan scan = new Scan(getBytes(startRow), getBytes(stopRow));
1514         addAttributes(scan, attributes);
1515         scan.setTimeRange(0, timestamp);
1516         if (columns != null && columns.size() != 0) {
1517           for (ByteBuffer column : columns) {
1518             byte [][] famQf = KeyValue.parseColumn(getBytes(column));
1519             if(famQf.length == 1) {
1520               scan.addFamily(famQf[0]);
1521             } else {
1522               scan.addColumn(famQf[0], famQf[1]);
1523             }
1524           }
1525         }
1526         scan.setTimeRange(0, timestamp);
1527         return addScanner(table.getScanner(scan), false);
1528       } catch (IOException e) {
1529         LOG.warn(e.getMessage(), e);
1530         throw new IOError(Throwables.getStackTraceAsString(e));
1531       } finally{
1532         closeTable(table);
1533       }
1534     }
1535 
1536     @Override
1537     public Map<ByteBuffer, ColumnDescriptor> getColumnDescriptors(
1538         ByteBuffer tableName) throws IOError, TException {
1539       HTableInterface table = null;
1540       try {
1541         TreeMap<ByteBuffer, ColumnDescriptor> columns =
1542           new TreeMap<ByteBuffer, ColumnDescriptor>();
1543 
1544         table = getTable(tableName);
1545         HTableDescriptor desc = table.getTableDescriptor();
1546 
1547         for (HColumnDescriptor e : desc.getFamilies()) {
1548           ColumnDescriptor col = ThriftUtilities.colDescFromHbase(e);
1549           columns.put(col.name, col);
1550         }
1551         return columns;
1552       } catch (IOException e) {
1553         LOG.warn(e.getMessage(), e);
1554         throw new IOError(Throwables.getStackTraceAsString(e));
1555       } finally {
1556         closeTable(table);
1557       }
1558     }
1559 
1560     @Override
1561     public List<TCell> getRowOrBefore(ByteBuffer tableName, ByteBuffer row,
1562         ByteBuffer family) throws IOError {
1563       HTableInterface table = null;
1564       try {
1565         table = getTable(getBytes(tableName));
1566         Result result = table.getRowOrBefore(getBytes(row), getBytes(family));
1567         return ThriftUtilities.cellFromHBase(result.rawCells());
1568       } catch (IOException e) {
1569         LOG.warn(e.getMessage(), e);
1570         throw new IOError(e.getMessage());
1571       } finally {
1572         closeTable(table);
1573       }
1574     }
1575 
1576     @Override
1577     public TRegionInfo getRegionInfo(ByteBuffer searchRow) throws IOError {
1578       HTableInterface table = null;
1579       try {
1580         table = getTable(TableName.META_TABLE_NAME.getName());
1581         byte[] row = getBytes(searchRow);
1582         Result startRowResult = table.getRowOrBefore(
1583           row, HConstants.CATALOG_FAMILY);
1584 
1585         if (startRowResult == null) {
1586           throw new IOException("Cannot find row in "+ TableName.META_TABLE_NAME+", row="
1587                                 + Bytes.toStringBinary(row));
1588         }
1589 
1590         // find region start and end keys
1591         HRegionInfo regionInfo = HRegionInfo.getHRegionInfo(startRowResult);
1592         if (regionInfo == null) {
1593           throw new IOException("HRegionInfo REGIONINFO was null or " +
1594                                 " empty in Meta for row="
1595                                 + Bytes.toStringBinary(row));
1596         }
1597         TRegionInfo region = new TRegionInfo();
1598         region.setStartKey(regionInfo.getStartKey());
1599         region.setEndKey(regionInfo.getEndKey());
1600         region.id = regionInfo.getRegionId();
1601         region.setName(regionInfo.getRegionName());
1602         region.version = regionInfo.getVersion();
1603 
1604         // find region assignment to server
1605         ServerName serverName = HRegionInfo.getServerName(startRowResult);
1606         if (serverName != null) {
1607           region.setServerName(Bytes.toBytes(serverName.getHostname()));
1608           region.port = serverName.getPort();
1609         }
1610         return region;
1611       } catch (IOException e) {
1612         LOG.warn(e.getMessage(), e);
1613         throw new IOError(Throwables.getStackTraceAsString(e));
1614       } finally {
1615         closeTable(table);
1616       }
1617     }
1618 
1619     private void closeTable(HTableInterface table) throws IOError
1620     {
1621       try{
1622         if(table != null){
1623           table.close();
1624         }
1625       } catch (IOException e){
1626         LOG.error(e.getMessage(), e);
1627         throw new IOError(Throwables.getStackTraceAsString(e));
1628       }
1629     }
1630 
1631     private void initMetrics(ThriftMetrics metrics) {
1632       this.metrics = metrics;
1633     }
1634 
1635     @Override
1636     public void increment(TIncrement tincrement) throws IOError, TException {
1637 
1638       if (tincrement.getRow().length == 0 || tincrement.getTable().length == 0) {
1639         throw new TException("Must supply a table and a row key; can't increment");
1640       }
1641 
1642       if (conf.getBoolean(COALESCE_INC_KEY, false)) {
1643         this.coalescer.queueIncrement(tincrement);
1644         return;
1645       }
1646 
1647       HTableInterface table = null;
1648       try {
1649         table = getTable(tincrement.getTable());
1650         Increment inc = ThriftUtilities.incrementFromThrift(tincrement);
1651         table.increment(inc);
1652       } catch (IOException e) {
1653         LOG.warn(e.getMessage(), e);
1654         throw new IOError(Throwables.getStackTraceAsString(e));
1655       } finally{
1656         closeTable(table);
1657       }
1658     }
1659 
1660     @Override
1661     public void incrementRows(List<TIncrement> tincrements) throws IOError, TException {
1662       if (conf.getBoolean(COALESCE_INC_KEY, false)) {
1663         this.coalescer.queueIncrements(tincrements);
1664         return;
1665       }
1666       for (TIncrement tinc : tincrements) {
1667         increment(tinc);
1668       }
1669     }
1670   }
1671 
1672   /**
1673    * Adds all the attributes into the Operation object
1674    */
1675   private static void addAttributes(OperationWithAttributes op,
1676     Map<ByteBuffer, ByteBuffer> attributes) {
1677     if (attributes == null || attributes.size() == 0) {
1678       return;
1679     }
1680     for (Map.Entry<ByteBuffer, ByteBuffer> entry : attributes.entrySet()) {
1681       String name = Bytes.toStringBinary(getBytes(entry.getKey()));
1682       byte[] value =  getBytes(entry.getValue());
1683       op.setAttribute(name, value);
1684     }
1685   }
1686 
1687   public static void registerFilters(Configuration conf) {
1688     String[] filters = conf.getStrings("hbase.thrift.filters");
1689     if(filters != null) {
1690       for(String filterClass: filters) {
1691         String[] filterPart = filterClass.split(":");
1692         if(filterPart.length != 2) {
1693           LOG.warn("Invalid filter specification " + filterClass + " - skipping");
1694         } else {
1695           ParseFilter.registerFilter(filterPart[0], filterPart[1]);
1696         }
1697       }
1698     }
1699   }
1700 }