001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019package org.apache.hadoop.hbase.master; 020 021import java.io.File; 022import java.io.IOException; 023import java.util.List; 024 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.hbase.HConstants; 027import org.apache.hadoop.hbase.LocalHBaseCluster; 028import org.apache.hadoop.hbase.MasterNotRunningException; 029import org.apache.hadoop.hbase.ZNodeClearer; 030import org.apache.hadoop.hbase.ZooKeeperConnectionException; 031import org.apache.hadoop.hbase.trace.TraceUtil; 032import org.apache.yetus.audience.InterfaceAudience; 033import org.apache.hadoop.hbase.client.Admin; 034import org.apache.hadoop.hbase.client.Connection; 035import org.apache.hadoop.hbase.client.ConnectionFactory; 036import org.apache.hadoop.hbase.regionserver.HRegionServer; 037import org.apache.hadoop.hbase.util.JVMClusterUtil; 038import org.apache.hadoop.hbase.util.ServerCommandLine; 039import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster; 040import org.apache.hadoop.hbase.zookeeper.ZKUtil; 041import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; 042import org.apache.zookeeper.KeeperException; 043import org.slf4j.Logger; 044import org.slf4j.LoggerFactory; 045 046import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; 047import org.apache.hbase.thirdparty.org.apache.commons.cli.GnuParser; 048import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; 049import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException; 050 051@InterfaceAudience.Private 052public class HMasterCommandLine extends ServerCommandLine { 053 private static final Logger LOG = LoggerFactory.getLogger(HMasterCommandLine.class); 054 055 private static final String USAGE = 056 "Usage: Master [opts] start|stop|clear\n" + 057 " start Start Master. If local mode, start Master and RegionServer in same JVM\n" + 058 " stop Start cluster shutdown; Master signals RegionServer shutdown\n" + 059 " clear Delete the master znode in ZooKeeper after a master crashes\n "+ 060 " where [opts] are:\n" + 061 " --minRegionServers=<servers> Minimum RegionServers needed to host user tables.\n" + 062 " --localRegionServers=<servers> " + 063 "RegionServers to start in master process when in standalone mode.\n" + 064 " --masters=<servers> Masters to start in this process.\n" + 065 " --backup Master should start in backup mode"; 066 067 private final Class<? extends HMaster> masterClass; 068 069 public HMasterCommandLine(Class<? extends HMaster> masterClass) { 070 this.masterClass = masterClass; 071 } 072 073 @Override 074 protected String getUsage() { 075 return USAGE; 076 } 077 078 @Override 079 public int run(String args[]) throws Exception { 080 Options opt = new Options(); 081 opt.addOption("localRegionServers", true, 082 "RegionServers to start in master process when running standalone"); 083 opt.addOption("masters", true, "Masters to start in this process"); 084 opt.addOption("minRegionServers", true, "Minimum RegionServers needed to host user tables"); 085 opt.addOption("backup", false, "Do not try to become HMaster until the primary fails"); 086 087 CommandLine cmd; 088 try { 089 cmd = new GnuParser().parse(opt, args); 090 } catch (ParseException e) { 091 LOG.error("Could not parse: ", e); 092 usage(null); 093 return 1; 094 } 095 096 097 if (cmd.hasOption("minRegionServers")) { 098 String val = cmd.getOptionValue("minRegionServers"); 099 getConf().setInt("hbase.regions.server.count.min", 100 Integer.parseInt(val)); 101 LOG.debug("minRegionServers set to " + val); 102 } 103 104 // minRegionServers used to be minServers. Support it too. 105 if (cmd.hasOption("minServers")) { 106 String val = cmd.getOptionValue("minServers"); 107 getConf().setInt("hbase.regions.server.count.min", Integer.parseInt(val)); 108 LOG.debug("minServers set to " + val); 109 } 110 111 // check if we are the backup master - override the conf if so 112 if (cmd.hasOption("backup")) { 113 getConf().setBoolean(HConstants.MASTER_TYPE_BACKUP, true); 114 } 115 116 // How many regionservers to startup in this process (we run regionservers in same process as 117 // master when we are in local/standalone mode. Useful testing) 118 if (cmd.hasOption("localRegionServers")) { 119 String val = cmd.getOptionValue("localRegionServers"); 120 getConf().setInt("hbase.regionservers", Integer.parseInt(val)); 121 LOG.debug("localRegionServers set to " + val); 122 } 123 // How many masters to startup inside this process; useful testing 124 if (cmd.hasOption("masters")) { 125 String val = cmd.getOptionValue("masters"); 126 getConf().setInt("hbase.masters", Integer.parseInt(val)); 127 LOG.debug("masters set to " + val); 128 } 129 130 @SuppressWarnings("unchecked") 131 List<String> remainingArgs = cmd.getArgList(); 132 if (remainingArgs.size() != 1) { 133 usage(null); 134 return 1; 135 } 136 137 String command = remainingArgs.get(0); 138 139 if ("start".equals(command)) { 140 return startMaster(); 141 } else if ("stop".equals(command)) { 142 return stopMaster(); 143 } else if ("clear".equals(command)) { 144 return (ZNodeClearer.clear(getConf()) ? 0 : 1); 145 } else { 146 usage("Invalid command: " + command); 147 return 1; 148 } 149 } 150 151 private int startMaster() { 152 Configuration conf = getConf(); 153 TraceUtil.initTracer(conf); 154 155 try { 156 // If 'local', defer to LocalHBaseCluster instance. Starts master 157 // and regionserver both in the one JVM. 158 if (LocalHBaseCluster.isLocal(conf)) { 159 DefaultMetricsSystem.setMiniClusterMode(true); 160 final MiniZooKeeperCluster zooKeeperCluster = new MiniZooKeeperCluster(conf); 161 File zkDataPath = new File(conf.get(HConstants.ZOOKEEPER_DATA_DIR)); 162 163 // find out the default client port 164 int zkClientPort = 0; 165 166 // If the zookeeper client port is specified in server quorum, use it. 167 String zkserver = conf.get(HConstants.ZOOKEEPER_QUORUM); 168 if (zkserver != null) { 169 String[] zkservers = zkserver.split(","); 170 171 if (zkservers.length > 1) { 172 // In local mode deployment, we have the master + a region server and zookeeper server 173 // started in the same process. Therefore, we only support one zookeeper server. 174 String errorMsg = "Could not start ZK with " + zkservers.length + 175 " ZK servers in local mode deployment. Aborting as clients (e.g. shell) will not " 176 + "be able to find this ZK quorum."; 177 System.err.println(errorMsg); 178 throw new IOException(errorMsg); 179 } 180 181 String[] parts = zkservers[0].split(":"); 182 183 if (parts.length == 2) { 184 // the second part is the client port 185 zkClientPort = Integer.parseInt(parts [1]); 186 } 187 } 188 // If the client port could not be find in server quorum conf, try another conf 189 if (zkClientPort == 0) { 190 zkClientPort = conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, 0); 191 // The client port has to be set by now; if not, throw exception. 192 if (zkClientPort == 0) { 193 throw new IOException("No config value for " + HConstants.ZOOKEEPER_CLIENT_PORT); 194 } 195 } 196 zooKeeperCluster.setDefaultClientPort(zkClientPort); 197 // set the ZK tick time if specified 198 int zkTickTime = conf.getInt(HConstants.ZOOKEEPER_TICK_TIME, 0); 199 if (zkTickTime > 0) { 200 zooKeeperCluster.setTickTime(zkTickTime); 201 } 202 203 // login the zookeeper server principal (if using security) 204 ZKUtil.loginServer(conf, HConstants.ZK_SERVER_KEYTAB_FILE, 205 HConstants.ZK_SERVER_KERBEROS_PRINCIPAL, null); 206 int localZKClusterSessionTimeout = 207 conf.getInt(HConstants.ZK_SESSION_TIMEOUT + ".localHBaseCluster", 10*1000); 208 conf.setInt(HConstants.ZK_SESSION_TIMEOUT, localZKClusterSessionTimeout); 209 LOG.info("Starting a zookeeper cluster"); 210 int clientPort = zooKeeperCluster.startup(zkDataPath); 211 if (clientPort != zkClientPort) { 212 String errorMsg = "Could not start ZK at requested port of " + 213 zkClientPort + ". ZK was started at port: " + clientPort + 214 ". Aborting as clients (e.g. shell) will not be able to find " + 215 "this ZK quorum."; 216 System.err.println(errorMsg); 217 throw new IOException(errorMsg); 218 } 219 conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, Integer.toString(clientPort)); 220 221 // Need to have the zk cluster shutdown when master is shutdown. 222 // Run a subclass that does the zk cluster shutdown on its way out. 223 int mastersCount = conf.getInt("hbase.masters", 1); 224 int regionServersCount = conf.getInt("hbase.regionservers", 1); 225 // Set start timeout to 5 minutes for cmd line start operations 226 conf.setIfUnset("hbase.master.start.timeout.localHBaseCluster", "300000"); 227 LOG.info("Starting up instance of localHBaseCluster; master=" + mastersCount + 228 ", regionserversCount=" + regionServersCount); 229 LocalHBaseCluster cluster = new LocalHBaseCluster(conf, mastersCount, regionServersCount, 230 LocalHMaster.class, HRegionServer.class); 231 ((LocalHMaster)cluster.getMaster(0)).setZKCluster(zooKeeperCluster); 232 cluster.startup(); 233 waitOnMasterThreads(cluster); 234 } else { 235 logProcessInfo(getConf()); 236 HMaster master = HMaster.constructMaster(masterClass, conf); 237 if (master.isStopped()) { 238 LOG.info("Won't bring the Master up as a shutdown is requested"); 239 return 1; 240 } 241 master.start(); 242 master.join(); 243 if(master.isAborted()) 244 throw new RuntimeException("HMaster Aborted"); 245 } 246 } catch (Throwable t) { 247 LOG.error("Master exiting", t); 248 return 1; 249 } 250 return 0; 251 } 252 253 @SuppressWarnings("resource") 254 private int stopMaster() { 255 Configuration conf = getConf(); 256 // Don't try more than once 257 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 0); 258 try (Connection connection = ConnectionFactory.createConnection(conf)) { 259 try (Admin admin = connection.getAdmin()) { 260 admin.shutdown(); 261 } catch (Throwable t) { 262 LOG.error("Failed to stop master", t); 263 return 1; 264 } 265 } catch (MasterNotRunningException e) { 266 LOG.error("Master not running"); 267 return 1; 268 } catch (ZooKeeperConnectionException e) { 269 LOG.error("ZooKeeper not available"); 270 return 1; 271 } catch (IOException e) { 272 LOG.error("Got IOException: " +e.getMessage(), e); 273 return 1; 274 } 275 return 0; 276 } 277 278 private void waitOnMasterThreads(LocalHBaseCluster cluster) throws InterruptedException{ 279 List<JVMClusterUtil.MasterThread> masters = cluster.getMasters(); 280 List<JVMClusterUtil.RegionServerThread> regionservers = cluster.getRegionServers(); 281 282 if (masters != null) { 283 for (JVMClusterUtil.MasterThread t : masters) { 284 t.join(); 285 if(t.getMaster().isAborted()) { 286 closeAllRegionServerThreads(regionservers); 287 throw new RuntimeException("HMaster Aborted"); 288 } 289 } 290 } 291 } 292 293 private static void closeAllRegionServerThreads( 294 List<JVMClusterUtil.RegionServerThread> regionservers) { 295 for(JVMClusterUtil.RegionServerThread t : regionservers){ 296 t.getRegionServer().stop("HMaster Aborted; Bringing down regions servers"); 297 } 298 } 299 300 /* 301 * Version of master that will shutdown the passed zk cluster on its way out. 302 */ 303 public static class LocalHMaster extends HMaster { 304 private MiniZooKeeperCluster zkcluster = null; 305 306 public LocalHMaster(Configuration conf) 307 throws IOException, KeeperException, InterruptedException { 308 super(conf); 309 } 310 311 @Override 312 public void run() { 313 super.run(); 314 if (this.zkcluster != null) { 315 try { 316 this.zkcluster.shutdown(); 317 } catch (IOException e) { 318 e.printStackTrace(); 319 } 320 } 321 } 322 323 void setZKCluster(final MiniZooKeeperCluster zkcluster) { 324 this.zkcluster = zkcluster; 325 } 326 } 327}