View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.thrift;
20  
21  import java.util.Arrays;
22  import java.util.List;
23  
24  import org.apache.commons.cli.CommandLine;
25  import org.apache.commons.cli.CommandLineParser;
26  import org.apache.commons.cli.HelpFormatter;
27  import org.apache.commons.cli.Options;
28  import org.apache.commons.cli.PosixParser;
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.hbase.HBaseConfiguration;
33  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.hbase.http.InfoServer;
36  import org.apache.hadoop.hbase.thrift.ThriftServerRunner.ImplType;
37  import org.apache.hadoop.hbase.util.VersionInfo;
38  import org.apache.hadoop.util.Shell.ExitCodeException;
39  
40  /**
41   * ThriftServer- this class starts up a Thrift server which implements the
42   * Hbase API specified in the Hbase.thrift IDL file. The server runs in an
43   * independent process.
44   */
45  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
46  public class ThriftServer {
47  
48    private static final Log LOG = LogFactory.getLog(ThriftServer.class);
49  
50    private static final String MIN_WORKERS_OPTION = "minWorkers";
51    private static final String MAX_WORKERS_OPTION = "workers";
52    private static final String MAX_QUEUE_SIZE_OPTION = "queue";
53    private static final String KEEP_ALIVE_SEC_OPTION = "keepAliveSec";
54    static final String BIND_OPTION = "bind";
55    static final String COMPACT_OPTION = "compact";
56    static final String FRAMED_OPTION = "framed";
57    static final String PORT_OPTION = "port";
58  
59    private static final String DEFAULT_BIND_ADDR = "0.0.0.0";
60    private static final int DEFAULT_LISTEN_PORT = 9090;
61  
62    private Configuration conf;
63    ThriftServerRunner serverRunner;
64  
65    private InfoServer infoServer;
66  
67    private static final String READ_TIMEOUT_OPTION = "readTimeout";
68  
69    //
70    // Main program and support routines
71    //
72  
73    public ThriftServer(Configuration conf) {
74      this.conf = HBaseConfiguration.create(conf);
75    }
76  
77    private static void printUsageAndExit(Options options, int exitCode)
78        throws ExitCodeException {
79      HelpFormatter formatter = new HelpFormatter();
80      formatter.printHelp("Thrift", null, options,
81          "To start the Thrift server run 'bin/hbase-daemon.sh start thrift'\n" +
82          "To shutdown the thrift server run 'bin/hbase-daemon.sh stop " +
83          "thrift' or send a kill signal to the thrift server pid",
84          true);
85      throw new ExitCodeException(exitCode, "");
86    }
87  
88    /**
89     * Start up or shuts down the Thrift server, depending on the arguments.
90     * @param args
91     */
92     void doMain(final String[] args) throws Exception {
93       processOptions(args);
94  
95       serverRunner = new ThriftServerRunner(conf);
96  
97       // Put up info server.
98       int port = conf.getInt("hbase.thrift.info.port", 9095);
99       if (port >= 0) {
100        conf.setLong("startcode", System.currentTimeMillis());
101        String a = conf.get("hbase.thrift.info.bindAddress", "0.0.0.0");
102        infoServer = new InfoServer("thrift", a, port, false, conf);
103        infoServer.setAttribute("hbase.conf", conf);
104        infoServer.start();
105      }
106      serverRunner.run();
107   }
108 
109   /**
110    * Parse the command line options to set parameters the conf.
111    */
112   private void processOptions(final String[] args) throws Exception {
113     Options options = new Options();
114     options.addOption("b", BIND_OPTION, true, "Address to bind " +
115         "the Thrift server to. [default: " + DEFAULT_BIND_ADDR + "]");
116     options.addOption("p", PORT_OPTION, true, "Port to bind to [default: " +
117         DEFAULT_LISTEN_PORT + "]");
118     options.addOption("f", FRAMED_OPTION, false, "Use framed transport");
119     options.addOption("c", COMPACT_OPTION, false, "Use the compact protocol");
120     options.addOption("h", "help", false, "Print help information");
121     options.addOption(null, "infoport", true, "Port for web UI");
122 
123     options.addOption("m", MIN_WORKERS_OPTION, true,
124         "The minimum number of worker threads for " +
125         ImplType.THREAD_POOL.simpleClassName());
126 
127     options.addOption("w", MAX_WORKERS_OPTION, true,
128         "The maximum number of worker threads for " +
129         ImplType.THREAD_POOL.simpleClassName());
130 
131     options.addOption("q", MAX_QUEUE_SIZE_OPTION, true,
132         "The maximum number of queued requests in " +
133         ImplType.THREAD_POOL.simpleClassName());
134 
135     options.addOption("k", KEEP_ALIVE_SEC_OPTION, true,
136         "The amount of time in secods to keep a thread alive when idle in " +
137         ImplType.THREAD_POOL.simpleClassName());
138 
139     options.addOption("t", READ_TIMEOUT_OPTION, true,
140         "Amount of time in milliseconds before a server thread will timeout " +
141         "waiting for client to send data on a connected socket. Currently, " +
142         "only applies to TBoundedThreadPoolServer");
143 
144     options.addOptionGroup(ImplType.createOptionGroup());
145 
146     CommandLineParser parser = new PosixParser();
147     CommandLine cmd = parser.parse(options, args);
148 
149     // This is so complicated to please both bin/hbase and bin/hbase-daemon.
150     // hbase-daemon provides "start" and "stop" arguments
151     // hbase should print the help if no argument is provided
152     List<String> commandLine = Arrays.asList(args);
153     boolean stop = commandLine.contains("stop");
154     boolean start = commandLine.contains("start");
155     boolean invalidStartStop = (start && stop) || (!start && !stop);
156     if (cmd.hasOption("help") || invalidStartStop) {
157       if (invalidStartStop) {
158         LOG.error("Exactly one of 'start' and 'stop' has to be specified");
159       }
160       printUsageAndExit(options, 1);
161     }
162 
163     // Get port to bind to
164     try {
165       if (cmd.hasOption(PORT_OPTION)) {
166         int listenPort = Integer.parseInt(cmd.getOptionValue(PORT_OPTION));
167         conf.setInt(ThriftServerRunner.PORT_CONF_KEY, listenPort);
168       }
169     } catch (NumberFormatException e) {
170       LOG.error("Could not parse the value provided for the port option", e);
171       printUsageAndExit(options, -1);
172     }
173 
174     // check for user-defined info server port setting, if so override the conf
175     try {
176       if (cmd.hasOption("infoport")) {
177         String val = cmd.getOptionValue("infoport");
178         conf.setInt("hbase.thrift.info.port", Integer.parseInt(val));
179         LOG.debug("Web UI port set to " + val);
180       }
181     } catch (NumberFormatException e) {
182       LOG.error("Could not parse the value provided for the infoport option", e);
183       printUsageAndExit(options, -1);
184     }
185 
186     // Make optional changes to the configuration based on command-line options
187     optionToConf(cmd, MIN_WORKERS_OPTION,
188         conf, TBoundedThreadPoolServer.MIN_WORKER_THREADS_CONF_KEY);
189     optionToConf(cmd, MAX_WORKERS_OPTION,
190         conf, TBoundedThreadPoolServer.MAX_WORKER_THREADS_CONF_KEY);
191     optionToConf(cmd, MAX_QUEUE_SIZE_OPTION,
192         conf, TBoundedThreadPoolServer.MAX_QUEUED_REQUESTS_CONF_KEY);
193     optionToConf(cmd, KEEP_ALIVE_SEC_OPTION,
194         conf, TBoundedThreadPoolServer.THREAD_KEEP_ALIVE_TIME_SEC_CONF_KEY);
195     optionToConf(cmd, READ_TIMEOUT_OPTION, conf,
196         ThriftServerRunner.THRIFT_SERVER_SOCKET_READ_TIMEOUT_KEY);
197     
198     // Set general thrift server options
199     boolean compact = cmd.hasOption(COMPACT_OPTION) ||
200       conf.getBoolean(ThriftServerRunner.COMPACT_CONF_KEY, false);
201     conf.setBoolean(ThriftServerRunner.COMPACT_CONF_KEY, compact);
202     boolean framed = cmd.hasOption(FRAMED_OPTION) ||
203       conf.getBoolean(ThriftServerRunner.FRAMED_CONF_KEY, false);
204     conf.setBoolean(ThriftServerRunner.FRAMED_CONF_KEY, framed);
205     if (cmd.hasOption(BIND_OPTION)) {
206       conf.set(ThriftServerRunner.BIND_CONF_KEY, cmd.getOptionValue(BIND_OPTION));
207     }
208 
209     ImplType.setServerImpl(cmd, conf);
210   }
211 
212   public void stop() {
213     if (this.infoServer != null) {
214       LOG.info("Stopping infoServer");
215       try {
216         this.infoServer.stop();
217       } catch (Exception ex) {
218         ex.printStackTrace();
219       }
220     }
221     serverRunner.shutdown();
222   }
223 
224   private static void optionToConf(CommandLine cmd, String option,
225       Configuration conf, String destConfKey) {
226     if (cmd.hasOption(option)) {
227       String value = cmd.getOptionValue(option);
228       LOG.info("Set configuration key:" + destConfKey + " value:" + value);
229       conf.set(destConfKey, value);
230     }
231   }
232 
233   /**
234    * @param args
235    * @throws Exception
236    */
237   public static void main(String [] args) throws Exception {
238     VersionInfo.logVersion();
239     try {
240       new ThriftServer(HBaseConfiguration.create()).doMain(args);
241     } catch (ExitCodeException ex) {
242       System.exit(ex.getExitCode());
243     }
244   }
245 }