View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import java.io.File;
23  import java.io.IOException;
24  import java.lang.management.ManagementFactory;
25  import java.lang.management.RuntimeMXBean;
26  import java.lang.reflect.Constructor;
27  import java.net.UnknownHostException;
28  import java.util.ArrayList;
29  import java.util.HashMap;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.NavigableMap;
33  import java.util.Set;
34  import java.util.SortedMap;
35  import java.util.concurrent.atomic.AtomicBoolean;
36  import java.util.concurrent.atomic.AtomicReference;
37  import java.util.concurrent.locks.Lock;
38  import java.util.concurrent.locks.ReentrantLock;
39  
40  import org.apache.commons.cli.CommandLine;
41  import org.apache.commons.cli.GnuParser;
42  import org.apache.commons.cli.Options;
43  import org.apache.commons.cli.ParseException;
44  import org.apache.commons.logging.Log;
45  import org.apache.commons.logging.LogFactory;
46  import org.apache.hadoop.conf.Configuration;
47  import org.apache.hadoop.fs.FileStatus;
48  import org.apache.hadoop.fs.FileSystem;
49  import org.apache.hadoop.fs.Path;
50  import org.apache.hadoop.hbase.ClusterStatus;
51  import org.apache.hadoop.hbase.HBaseConfiguration;
52  import org.apache.hadoop.hbase.HColumnDescriptor;
53  import org.apache.hadoop.hbase.HConstants;
54  import org.apache.hadoop.hbase.HMsg;
55  import org.apache.hadoop.hbase.HRegionInfo;
56  import org.apache.hadoop.hbase.HRegionLocation;
57  import org.apache.hadoop.hbase.HServerAddress;
58  import org.apache.hadoop.hbase.HServerInfo;
59  import org.apache.hadoop.hbase.HServerLoad;
60  import org.apache.hadoop.hbase.HTableDescriptor;
61  import org.apache.hadoop.hbase.KeyValue;
62  import org.apache.hadoop.hbase.LocalHBaseCluster;
63  import org.apache.hadoop.hbase.MasterNotRunningException;
64  import org.apache.hadoop.hbase.MiniZooKeeperCluster;
65  import org.apache.hadoop.hbase.RemoteExceptionHandler;
66  import org.apache.hadoop.hbase.TableExistsException;
67  import org.apache.hadoop.hbase.client.Get;
68  import org.apache.hadoop.hbase.client.HBaseAdmin;
69  import org.apache.hadoop.hbase.client.Result;
70  import org.apache.hadoop.hbase.client.MetaScanner;
71  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
72  import org.apache.hadoop.hbase.client.Scan;
73  import org.apache.hadoop.hbase.client.ServerConnection;
74  import org.apache.hadoop.hbase.client.ServerConnectionManager;
75  import org.apache.hadoop.hbase.executor.HBaseEventHandler;
76  import org.apache.hadoop.hbase.executor.HBaseExecutorService;
77  import org.apache.hadoop.hbase.executor.HBaseEventHandler.HBaseEventType;
78  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
79  import org.apache.hadoop.hbase.ipc.HBaseRPC;
80  import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
81  import org.apache.hadoop.hbase.ipc.HBaseServer;
82  import org.apache.hadoop.hbase.ipc.HMasterInterface;
83  import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
84  import org.apache.hadoop.hbase.ipc.HRegionInterface;
85  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
86  import org.apache.hadoop.hbase.regionserver.HRegion;
87  import org.apache.hadoop.hbase.regionserver.HRegionServer;
88  import org.apache.hadoop.hbase.regionserver.wal.HLog;
89  import org.apache.hadoop.hbase.util.Bytes;
90  import org.apache.hadoop.hbase.util.FSUtils;
91  import org.apache.hadoop.hbase.util.InfoServer;
92  import org.apache.hadoop.hbase.util.Pair;
93  import org.apache.hadoop.hbase.util.Sleeper;
94  import org.apache.hadoop.hbase.util.VersionInfo;
95  import org.apache.hadoop.hbase.util.Writables;
96  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
97  import org.apache.hadoop.io.MapWritable;
98  import org.apache.hadoop.io.Text;
99  import org.apache.hadoop.io.Writable;
100 import org.apache.hadoop.ipc.RemoteException;
101 import org.apache.hadoop.net.DNS;
102 import org.apache.zookeeper.WatchedEvent;
103 import org.apache.zookeeper.Watcher;
104 import org.apache.zookeeper.Watcher.Event.EventType;
105 import org.apache.zookeeper.Watcher.Event.KeeperState;
106 
107 import com.google.common.collect.Lists;
108 
109 /**
110  * HMaster is the "master server" for HBase. An HBase cluster has one active
111  * master.  If many masters are started, all compete.  Whichever wins goes on to
112  * run the cluster.  All others park themselves in their constructor until
113  * master or cluster shutdown or until the active master loses its lease in
114  * zookeeper.  Thereafter, all running master jostle to take over master role.
115  * @see HMasterInterface
116  * @see HMasterRegionInterface
117  * @see Watcher
118  */
119 public class HMaster extends Thread implements HMasterInterface,
120     HMasterRegionInterface, Watcher {
121   // MASTER is name of the webapp and the attribute name used stuffing this
122   //instance into web context.
123   public static final String MASTER = "master";
124   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
125 
126   // We start out with closed flag on.  Its set to off after construction.
127   // Use AtomicBoolean rather than plain boolean because we want other threads
128   // able to set shutdown flag.  Using AtomicBoolean can pass a reference
129   // rather than have them have to know about the hosting Master class.
130   final AtomicBoolean closed = new AtomicBoolean(true);
131   // TODO: Is this separate flag necessary?
132   private final AtomicBoolean shutdownRequested = new AtomicBoolean(false);
133 
134   private final Configuration conf;
135   private final Path rootdir;
136   private InfoServer infoServer;
137   private final int threadWakeFrequency;
138   private final int numRetries;
139 
140   // Metrics is set when we call run.
141   private final MasterMetrics metrics;
142 
143   final Lock splitLogLock = new ReentrantLock();
144 
145   // Our zk client.
146   private ZooKeeperWrapper zooKeeperWrapper;
147   // Watcher for master address and for cluster shutdown.
148   private final ZKMasterAddressWatcher zkMasterAddressWatcher;
149   // A Sleeper that sleeps for threadWakeFrequency; sleep if nothing todo.
150   private final Sleeper sleeper;
151   // Keep around for convenience.
152   private final FileSystem fs;
153   // Is the fileystem ok?
154   private volatile boolean fsOk = true;
155   // The Path to the old logs dir
156   private final Path oldLogDir;
157 
158   private final HBaseServer rpcServer;
159   private final HServerAddress address;
160 
161   private final ServerConnection connection;
162   private final ServerManager serverManager;
163   private final RegionManager regionManager;
164 
165   private long lastFragmentationQuery = -1L;
166   private Map<String, Integer> fragmentation = null;
167   private final RegionServerOperationQueue regionServerOperationQueue;
168   
169   // True if this is the master that started the cluster.
170   boolean isClusterStartup;
171 
172   /**
173    * Constructor
174    * @param conf configuration
175    * @throws IOException
176    */
177   public HMaster(Configuration conf) throws IOException {
178     this.conf = conf;
179     
180     // Figure out if this is a fresh cluster start. This is done by checking the 
181     // number of RS ephemeral nodes. RS ephemeral nodes are created only after 
182     // the primary master has written the address to ZK. So this has to be done 
183     // before we race to write our address to zookeeper.
184     zooKeeperWrapper = ZooKeeperWrapper.createInstance(conf, HMaster.class.getName());
185     isClusterStartup = (zooKeeperWrapper.scanRSDirectory().size() == 0);
186     
187     // Get my address and create an rpc server instance.  The rpc-server port
188     // can be ephemeral...ensure we have the correct info
189     HServerAddress a = new HServerAddress(getMyAddress(this.conf));
190     this.rpcServer = HBaseRPC.getServer(this, a.getBindAddress(),
191       a.getPort(), conf.getInt("hbase.regionserver.handler.count", 10),
192       false, conf);
193     this.address = new HServerAddress(this.rpcServer.getListenerAddress());
194 
195     this.numRetries =  conf.getInt("hbase.client.retries.number", 2);
196     this.threadWakeFrequency = conf.getInt(HConstants.THREAD_WAKE_FREQUENCY,
197         10 * 1000);
198 
199     this.sleeper = new Sleeper(this.threadWakeFrequency, this.closed);
200     this.connection = ServerConnectionManager.getConnection(conf);
201 
202     // hack! Maps DFSClient => Master for logs.  HDFS made this 
203     // config param for task trackers, but we can piggyback off of it.
204     if (this.conf.get("mapred.task.id") == null) {
205       this.conf.set("mapred.task.id", "hb_m_" + this.address.toString());
206     }
207 
208     // Set filesystem to be that of this.rootdir else we get complaints about
209     // mismatched filesystems if hbase.rootdir is hdfs and fs.defaultFS is
210     // default localfs.  Presumption is that rootdir is fully-qualified before
211     // we get to here with appropriate fs scheme.
212     this.rootdir = FSUtils.getRootDir(this.conf);
213     // Cover both bases, the old way of setting default fs and the new.
214     // We're supposed to run on 0.20 and 0.21 anyways.
215     this.conf.set("fs.default.name", this.rootdir.toString());
216     this.conf.set("fs.defaultFS", this.rootdir.toString());
217     this.fs = FileSystem.get(this.conf);
218     checkRootDir(this.rootdir, this.conf, this.fs);
219 
220     // Make sure the region servers can archive their old logs
221     this.oldLogDir = new Path(this.rootdir, HConstants.HREGION_OLDLOGDIR_NAME);
222     if(!this.fs.exists(this.oldLogDir)) {
223       this.fs.mkdirs(this.oldLogDir);
224     }
225 
226     // Get our zookeeper wrapper and then try to write our address to zookeeper.
227     // We'll succeed if we are only  master or if we win the race when many
228     // masters.  Otherwise we park here inside in writeAddressToZooKeeper.
229     // TODO: Bring up the UI to redirect to active Master.
230     zooKeeperWrapper.registerListener(this);
231     this.zkMasterAddressWatcher =
232       new ZKMasterAddressWatcher(this.zooKeeperWrapper, this.shutdownRequested);
233     zooKeeperWrapper.registerListener(zkMasterAddressWatcher);
234     this.zkMasterAddressWatcher.writeAddressToZooKeeper(this.address, true);
235     this.regionServerOperationQueue =
236       new RegionServerOperationQueue(this.conf, this.closed);
237 
238     serverManager = new ServerManager(this);
239 
240     
241     // Start the unassigned watcher - which will create the unassigned region 
242     // in ZK. This is needed before RegionManager() constructor tries to assign 
243     // the root region.
244     ZKUnassignedWatcher.start(this.conf, this);
245     // start the "close region" executor service
246     HBaseEventType.RS2ZK_REGION_CLOSED.startMasterExecutorService(address.toString());
247     // start the "open region" executor service
248     HBaseEventType.RS2ZK_REGION_OPENED.startMasterExecutorService(address.toString());
249 
250     
251     // start the region manager
252     regionManager = new RegionManager(this);
253 
254     setName(MASTER);
255     this.metrics = new MasterMetrics(MASTER);
256     // We're almost open for business
257     this.closed.set(false);
258     LOG.info("HMaster initialized on " + this.address.toString());
259   }
260   
261   /**
262    * Returns true if this master process was responsible for starting the 
263    * cluster.
264    */
265   public boolean isClusterStartup() {
266     return isClusterStartup;
267   }
268   
269   public void resetClusterStartup() {
270     isClusterStartup = false;
271   }
272   
273   public HServerAddress getHServerAddress() {
274     return address;
275   }
276 
277   /*
278    * Get the rootdir.  Make sure its wholesome and exists before returning.
279    * @param rd
280    * @param conf
281    * @param fs
282    * @return hbase.rootdir (after checks for existence and bootstrapping if
283    * needed populating the directory with necessary bootup files).
284    * @throws IOException
285    */
286   private static Path checkRootDir(final Path rd, final Configuration c,
287     final FileSystem fs)
288   throws IOException {
289     // If FS is in safe mode wait till out of it.
290     FSUtils.waitOnSafeMode(c, c.getInt(HConstants.THREAD_WAKE_FREQUENCY,
291         10 * 1000));
292     // Filesystem is good. Go ahead and check for hbase.rootdir.
293     if (!fs.exists(rd)) {
294       fs.mkdirs(rd);
295       FSUtils.setVersion(fs, rd);
296     } else {
297       FSUtils.checkVersion(fs, rd, true);
298     }
299     // Make sure the root region directory exists!
300     if (!FSUtils.rootRegionExists(fs, rd)) {
301       bootstrap(rd, c);
302     }
303     return rd;
304   }
305 
306   private static void bootstrap(final Path rd, final Configuration c)
307   throws IOException {
308     LOG.info("BOOTSTRAP: creating ROOT and first META regions");
309     try {
310       // Bootstrapping, make sure blockcache is off.  Else, one will be
311       // created here in bootstap and it'll need to be cleaned up.  Better to
312       // not make it in first place.  Turn off block caching for bootstrap.
313       // Enable after.
314       HRegionInfo rootHRI = new HRegionInfo(HRegionInfo.ROOT_REGIONINFO);
315       setInfoFamilyCaching(rootHRI, false);
316       HRegionInfo metaHRI = new HRegionInfo(HRegionInfo.FIRST_META_REGIONINFO);
317       setInfoFamilyCaching(metaHRI, false);
318       HRegion root = HRegion.createHRegion(rootHRI, rd, c);
319       HRegion meta = HRegion.createHRegion(metaHRI, rd, c);
320       setInfoFamilyCaching(rootHRI, true);
321       setInfoFamilyCaching(metaHRI, true);
322       // Add first region from the META table to the ROOT region.
323       HRegion.addRegionToMETA(root, meta);
324       root.close();
325       root.getLog().closeAndDelete();
326       meta.close();
327       meta.getLog().closeAndDelete();
328     } catch (IOException e) {
329       e = RemoteExceptionHandler.checkIOException(e);
330       LOG.error("bootstrap", e);
331       throw e;
332     }
333   }
334 
335   /*
336    * @param hri Set all family block caching to <code>b</code>
337    * @param b
338    */
339   private static void setInfoFamilyCaching(final HRegionInfo hri, final boolean b) {
340     for (HColumnDescriptor hcd: hri.getTableDesc().families.values()) {
341       if (Bytes.equals(hcd.getName(), HConstants.CATALOG_FAMILY)) {
342         hcd.setBlockCacheEnabled(b);
343         hcd.setInMemory(b);
344       }
345     }
346   }
347 
348   /*
349    * @return This masters' address.
350    * @throws UnknownHostException
351    */
352   private static String getMyAddress(final Configuration c)
353   throws UnknownHostException {
354     // Find out our address up in DNS.
355     String s = DNS.getDefaultHost(c.get("hbase.master.dns.interface","default"),
356       c.get("hbase.master.dns.nameserver","default"));
357     s += ":" + c.get(HConstants.MASTER_PORT,
358         Integer.toString(HConstants.DEFAULT_MASTER_PORT));
359     return s;
360   }
361 
362   /**
363    * Checks to see if the file system is still accessible.
364    * If not, sets closed
365    * @return false if file system is not available
366    */
367   protected boolean checkFileSystem() {
368     if (this.fsOk) {
369       try {
370         FSUtils.checkFileSystemAvailable(this.fs);
371       } catch (IOException e) {
372         LOG.fatal("Shutting down HBase cluster: file system not available", e);
373         this.closed.set(true);
374         this.fsOk = false;
375       }
376     }
377     return this.fsOk;
378   }
379 
380   /** @return HServerAddress of the master server */
381   public HServerAddress getMasterAddress() {
382     return this.address;
383   }
384 
385   public long getProtocolVersion(String protocol, long clientVersion) {
386     return HBaseRPCProtocolVersion.versionID;
387   }
388 
389   /** @return InfoServer object. Maybe null.*/
390   public InfoServer getInfoServer() {
391     return this.infoServer;
392   }
393 
394   /**
395    * @return HBase root dir.
396    * @throws IOException
397    */
398   public Path getRootDir() {
399     return this.rootdir;
400   }
401 
402   public int getNumRetries() {
403     return this.numRetries;
404   }
405 
406   /**
407    * @return Server metrics
408    */
409   public MasterMetrics getMetrics() {
410     return this.metrics;
411   }
412 
413   /**
414    * @return Return configuration being used by this server.
415    */
416   public Configuration getConfiguration() {
417     return this.conf;
418   }
419 
420   public ServerManager getServerManager() {
421     return this.serverManager;
422   }
423 
424   public RegionManager getRegionManager() {
425     return this.regionManager;
426   }
427 
428   int getThreadWakeFrequency() {
429     return this.threadWakeFrequency;
430   }
431 
432   FileSystem getFileSystem() {
433     return this.fs;
434   }
435 
436   AtomicBoolean getShutdownRequested() {
437     return this.shutdownRequested;
438   }
439 
440   AtomicBoolean getClosed() {
441     return this.closed;
442   }
443 
444   boolean isClosed() {
445     return this.closed.get();
446   }
447 
448   ServerConnection getServerConnection() {
449     return this.connection;
450   }
451 
452   /**
453    * Get the ZK wrapper object
454    * @return the zookeeper wrapper
455    */
456   public ZooKeeperWrapper getZooKeeperWrapper() {
457     return this.zooKeeperWrapper;
458   }
459 
460   // These methods are so don't have to pollute RegionManager with ServerManager.
461   SortedMap<HServerLoad, Set<String>> getLoadToServers() {
462     return this.serverManager.getLoadToServers();
463   }
464 
465   int numServers() {
466     return this.serverManager.numServers();
467   }
468 
469   public double getAverageLoad() {
470     return this.serverManager.getAverageLoad();
471   }
472 
473   public RegionServerOperationQueue getRegionServerOperationQueue () {
474     return this.regionServerOperationQueue;
475   }
476 
477   /**
478    * Get the directory where old logs go
479    * @return the dir
480    */
481   public Path getOldLogDir() {
482     return this.oldLogDir;
483   }
484 
485   /**
486    * Add to the passed <code>m</code> servers that are loaded less than
487    * <code>l</code>.
488    * @param l
489    * @param m
490    */
491   void getLightServers(final HServerLoad l,
492       SortedMap<HServerLoad, Set<String>> m) {
493     this.serverManager.getLightServers(l, m);
494   }
495 
496   /** Main processing loop */
497   @Override
498   public void run() {
499     joinCluster();
500     startServiceThreads();
501     /* Main processing loop */
502     try {
503       FINISHED: while (!this.closed.get()) {
504         // check if we should be shutting down
505         if (this.shutdownRequested.get()) {
506           // The region servers won't all exit until we stop scanning the
507           // meta regions
508           this.regionManager.stopScanners();
509           if (this.serverManager.numServers() == 0) {
510             startShutdown();
511             break;
512           } else {
513             LOG.debug("Waiting on " +
514               this.serverManager.getServersToServerInfo().keySet().toString());
515           }
516         }
517         switch (this.regionServerOperationQueue.process()) {
518         case FAILED:
519             // If FAILED op processing, bad. Exit.
520           break FINISHED;
521         case REQUEUED_BUT_PROBLEM:
522           if (!checkFileSystem())
523               // If bad filesystem, exit.
524             break FINISHED;
525           default:
526             // Continue run loop if conditions are PROCESSED, NOOP, REQUEUED
527           break;
528         }
529       }
530     } catch (Throwable t) {
531       LOG.fatal("Unhandled exception. Starting shutdown.", t);
532       this.closed.set(true);
533     }
534 
535     // Wait for all the remaining region servers to report in.
536     this.serverManager.letRegionServersShutdown();
537 
538     /*
539      * Clean up and close up shop
540      */
541     if (this.infoServer != null) {
542       LOG.info("Stopping infoServer");
543       try {
544         this.infoServer.stop();
545       } catch (Exception ex) {
546         ex.printStackTrace();
547       }
548     }
549     this.rpcServer.stop();
550     this.regionManager.stop();
551     this.zooKeeperWrapper.close();
552     HBaseExecutorService.shutdown();
553     LOG.info("HMaster main thread exiting");
554   }
555 
556   /*
557    * Joins cluster.  Checks to see if this instance of HBase is fresh or the
558    * master was started following a failover. In the second case, it inspects
559    * the region server directory and gets their regions assignment.
560    */
561   private void joinCluster()  {
562       LOG.debug("Checking cluster state...");
563       HServerAddress rootLocation =
564         this.zooKeeperWrapper.readRootRegionLocation();
565       List<HServerAddress> addresses = this.zooKeeperWrapper.scanRSDirectory();
566       // Check if this is a fresh start of the cluster
567       if (addresses.isEmpty()) {
568         LOG.debug("Master fresh start, proceeding with normal startup");
569         splitLogAfterStartup();
570         return;
571       }
572       // Failover case.
573       LOG.info("Master failover, ZK inspection begins...");
574       boolean isRootRegionAssigned = false;
575       Map <byte[], HRegionInfo> assignedRegions =
576         new HashMap<byte[], HRegionInfo>();
577       // We must:
578       // - contact every region server to add them to the regionservers list
579       // - get their current regions assignment
580       // TODO: Run in parallel?
581       for (HServerAddress address : addresses) {
582         HRegionInfo[] regions = null;
583         try {
584           HRegionInterface hri =
585             this.connection.getHRegionConnection(address, false);
586           HServerInfo info = hri.getHServerInfo();
587           LOG.debug("Inspection found server " + info.getServerName());
588           this.serverManager.recordNewServer(info, true);
589           regions = hri.getRegionsAssignment();
590         } catch (IOException e) {
591           LOG.error("Failed contacting " + address.toString(), e);
592           continue;
593         }
594         for (HRegionInfo r: regions) {
595           if (r.isRootRegion()) {
596             this.connection.setRootRegionLocation(new HRegionLocation(r, rootLocation));
597             this.regionManager.setRootRegionLocation(rootLocation);
598             // Undo the unassign work in the RegionManager constructor
599             this.regionManager.removeRegion(r);
600             isRootRegionAssigned = true;
601           } else if (r.isMetaRegion()) {
602             MetaRegion m = new MetaRegion(new HServerAddress(address), r);
603             this.regionManager.addMetaRegionToScan(m);
604           }
605           assignedRegions.put(r.getRegionName(), r);
606         }
607       }
608       LOG.info("Inspection found " + assignedRegions.size() + " regions, " +
609         (isRootRegionAssigned ? "with -ROOT-" : "but -ROOT- was MIA"));
610       splitLogAfterStartup();
611   }
612 
613   /*
614    * Inspect the log directory to recover any log file without
615    * ad active region server.
616    */
617   private void splitLogAfterStartup() {
618     Path logsDirPath =
619       new Path(this.rootdir, HConstants.HREGION_LOGDIR_NAME);
620     try {
621       if (!this.fs.exists(logsDirPath)) return;
622     } catch (IOException e) {
623       throw new RuntimeException("Could exists for " + logsDirPath, e);
624     }
625     FileStatus[] logFolders;
626     try {
627       logFolders = this.fs.listStatus(logsDirPath);
628     } catch (IOException e) {
629       throw new RuntimeException("Failed listing " + logsDirPath.toString(), e);
630     }
631     if (logFolders == null || logFolders.length == 0) {
632       LOG.debug("No log files to split, proceeding...");
633       return;
634     }
635     for (FileStatus status : logFolders) {
636       String serverName = status.getPath().getName();
637       LOG.info("Found log folder : " + serverName);
638       if(this.serverManager.getServerInfo(serverName) == null) {
639         LOG.info("Log folder doesn't belong " +
640           "to a known region server, splitting");
641         this.splitLogLock.lock();
642         Path logDir =
643           new Path(this.rootdir, HLog.getHLogDirectoryName(serverName));
644         try {
645           HLog.splitLog(this.rootdir, logDir, oldLogDir, this.fs, getConfiguration());
646         } catch (IOException e) {
647           LOG.error("Failed splitting " + logDir.toString(), e);
648         } finally {
649           this.splitLogLock.unlock();
650         }
651       } else {
652         LOG.info("Log folder belongs to an existing region server");
653       }
654     }
655   }
656 
657   /*
658    * Start up all services. If any of these threads gets an unhandled exception
659    * then they just die with a logged message.  This should be fine because
660    * in general, we do not expect the master to get such unhandled exceptions
661    *  as OOMEs; it should be lightly loaded. See what HRegionServer does if
662    *  need to install an unexpected exception handler.
663    */
664   private void startServiceThreads() {
665     try {
666       this.regionManager.start();
667       // Put up info server.
668       int port = this.conf.getInt("hbase.master.info.port", 60010);
669       if (port >= 0) {
670         String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
671         this.infoServer = new InfoServer(MASTER, a, port, false);
672         this.infoServer.setAttribute(MASTER, this);
673         this.infoServer.start();
674       }
675       // Start the server so everything else is running before we start
676       // receiving requests.
677       this.rpcServer.start();
678       if (LOG.isDebugEnabled()) {
679         LOG.debug("Started service threads");
680       }
681     } catch (IOException e) {
682       if (e instanceof RemoteException) {
683         try {
684           e = RemoteExceptionHandler.decodeRemoteException((RemoteException) e);
685         } catch (IOException ex) {
686           LOG.warn("thread start", ex);
687         }
688       }
689       // Something happened during startup. Shut things down.
690       this.closed.set(true);
691       LOG.error("Failed startup", e);
692     }
693   }
694 
695   /*
696    * Start shutting down the master
697    */
698   void startShutdown() {
699     this.closed.set(true);
700     this.regionManager.stopScanners();
701     this.regionServerOperationQueue.shutdown();
702     this.serverManager.notifyServers();
703   }
704 
705   public MapWritable regionServerStartup(final HServerInfo serverInfo)
706   throws IOException {
707     // Set the ip into the passed in serverInfo.  Its ip is more than likely
708     // not the ip that the master sees here.  See at end of this method where
709     // we pass it back to the regionserver by setting "hbase.regionserver.address"
710     String rsAddress = HBaseServer.getRemoteAddress();
711     serverInfo.setServerAddress(new HServerAddress(rsAddress,
712       serverInfo.getServerAddress().getPort()));
713     // Register with server manager
714     this.serverManager.regionServerStartup(serverInfo);
715     // Send back some config info
716     MapWritable mw = createConfigurationSubset();
717      mw.put(new Text("hbase.regionserver.address"), new Text(rsAddress));
718     return mw;
719   }
720 
721   /**
722    * @return Subset of configuration to pass initializing regionservers: e.g.
723    * the filesystem to use and root directory to use.
724    */
725   protected MapWritable createConfigurationSubset() {
726     MapWritable mw = addConfig(new MapWritable(), HConstants.HBASE_DIR);
727     return addConfig(mw, "fs.default.name");
728   }
729 
730   private MapWritable addConfig(final MapWritable mw, final String key) {
731     mw.put(new Text(key), new Text(this.conf.get(key)));
732     return mw;
733   }
734 
735   public HMsg [] regionServerReport(HServerInfo serverInfo, HMsg msgs[],
736     HRegionInfo[] mostLoadedRegions)
737   throws IOException {
738     return adornRegionServerAnswer(serverInfo,
739       this.serverManager.regionServerReport(serverInfo, msgs, mostLoadedRegions));
740   }
741 
742   /**
743    * Override if you'd add messages to return to regionserver <code>hsi</code>
744    * or to send an exception.
745    * @param msgs Messages to add to
746    * @return Messages to return to
747    * @throws IOException exceptions that were injected for the region servers
748    */
749   protected HMsg [] adornRegionServerAnswer(final HServerInfo hsi,
750       final HMsg [] msgs) throws IOException {
751     return msgs;
752   }
753 
754   public boolean isMasterRunning() {
755     return !this.closed.get();
756   }
757 
758   public void shutdown() {
759     LOG.info("Cluster shutdown requested. Starting to quiesce servers");
760     this.shutdownRequested.set(true);
761     this.zooKeeperWrapper.setClusterState(false);
762   }
763 
764   public void createTable(HTableDescriptor desc, byte [][] splitKeys)
765   throws IOException {
766     if (!isMasterRunning()) {
767       throw new MasterNotRunningException();
768     }
769     HRegionInfo [] newRegions = null;
770     if(splitKeys == null || splitKeys.length == 0) {
771       newRegions = new HRegionInfo [] { new HRegionInfo(desc, null, null) };
772     } else {
773       int numRegions = splitKeys.length + 1;
774       newRegions = new HRegionInfo[numRegions];
775       byte [] startKey = null;
776       byte [] endKey = null;
777       for(int i=0;i<numRegions;i++) {
778         endKey = (i == splitKeys.length) ? null : splitKeys[i];
779         newRegions[i] = new HRegionInfo(desc, startKey, endKey);
780         startKey = endKey;
781       }
782     }
783     for (int tries = 0; tries < this.numRetries; tries++) {
784       try {
785         // We can not create a table unless meta regions have already been
786         // assigned and scanned.
787         if (!this.regionManager.areAllMetaRegionsOnline()) {
788           throw new NotAllMetaRegionsOnlineException();
789         }
790         if (!this.serverManager.canAssignUserRegions()) {
791           throw new IOException("not enough servers to create table yet");
792         }
793         createTable(newRegions);
794         LOG.info("created table " + desc.getNameAsString());
795         break;
796       } catch (TableExistsException e) {
797         throw e;
798       } catch (IOException e) {
799         if (tries == this.numRetries - 1) {
800           throw RemoteExceptionHandler.checkIOException(e);
801         }
802         this.sleeper.sleep();
803       }
804     }
805   }
806 
807   private synchronized void createTable(final HRegionInfo [] newRegions)
808   throws IOException {
809     String tableName = newRegions[0].getTableDesc().getNameAsString();
810     // 1. Check to see if table already exists. Get meta region where
811     // table would sit should it exist. Open scanner on it. If a region
812     // for the table we want to create already exists, then table already
813     // created. Throw already-exists exception.
814     MetaRegion m = regionManager.getFirstMetaRegionForRegion(newRegions[0]);
815     byte [] metaRegionName = m.getRegionName();
816     HRegionInterface srvr = this.connection.getHRegionConnection(m.getServer());
817     byte[] firstRowInTable = Bytes.toBytes(tableName + ",,");
818     Scan scan = new Scan(firstRowInTable);
819     scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
820     long scannerid = srvr.openScanner(metaRegionName, scan);
821     try {
822       Result data = srvr.next(scannerid);
823       if (data != null && data.size() > 0) {
824         HRegionInfo info = Writables.getHRegionInfo(
825           data.getValue(HConstants.CATALOG_FAMILY,
826               HConstants.REGIONINFO_QUALIFIER));
827         if (info.getTableDesc().getNameAsString().equals(tableName)) {
828           // A region for this table already exists. Ergo table exists.
829           throw new TableExistsException(tableName);
830         }
831       }
832     } finally {
833       srvr.close(scannerid);
834     }
835     for(HRegionInfo newRegion : newRegions) {
836       regionManager.createRegion(newRegion, srvr, metaRegionName);
837     }
838   }
839 
840   public void deleteTable(final byte [] tableName) throws IOException {
841     if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
842       throw new IOException("Can't delete root table");
843     }
844     new TableDelete(this, tableName).process();
845     LOG.info("deleted table: " + Bytes.toString(tableName));
846   }
847 
848   public void addColumn(byte [] tableName, HColumnDescriptor column)
849   throws IOException {
850     new AddColumn(this, tableName, column).process();
851   }
852 
853   public void modifyColumn(byte [] tableName, byte [] columnName,
854     HColumnDescriptor descriptor)
855   throws IOException {
856     new ModifyColumn(this, tableName, columnName, descriptor).process();
857   }
858 
859   public void deleteColumn(final byte [] tableName, final byte [] c)
860   throws IOException {
861     new DeleteColumn(this, tableName, KeyValue.parseColumn(c)[0]).process();
862   }
863 
864   public void enableTable(final byte [] tableName) throws IOException {
865     if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
866       throw new IOException("Can't enable root table");
867     }
868     new ChangeTableState(this, tableName, true).process();
869   }
870 
871   public void disableTable(final byte [] tableName) throws IOException {
872     if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME)) {
873       throw new IOException("Can't disable root table");
874     }
875     new ChangeTableState(this, tableName, false).process();
876   }
877 
878   /**
879    * Get a list of the regions for a given table. The pairs may have
880    * null for their second element in the case that they are not
881    * currently deployed.
882    * TODO: Redo so this method does not duplicate code with subsequent methods.
883    */
884   List<Pair<HRegionInfo,HServerAddress>> getTableRegions(
885       final byte [] tableName)
886   throws IOException {
887     final ArrayList<Pair<HRegionInfo, HServerAddress>> result =
888       Lists.newArrayList();
889     MetaScannerVisitor visitor =
890       new MetaScannerVisitor() {
891         @Override
892         public boolean processRow(Result data) throws IOException {
893           if (data == null || data.size() <= 0)
894             return true;
895           Pair<HRegionInfo, HServerAddress> pair =
896             metaRowToRegionPair(data);
897           if (pair == null) return false;
898           if (!Bytes.equals(pair.getFirst().getTableDesc().getName(),
899                 tableName)) {
900             return false;
901           }
902           result.add(pair);
903           return true;
904         }
905     };
906 
907     MetaScanner.metaScan(conf, visitor, tableName); 
908     return result;
909   }
910   
911   private Pair<HRegionInfo, HServerAddress> metaRowToRegionPair(
912       Result data) throws IOException {
913     HRegionInfo info = Writables.getHRegionInfo(
914         data.getValue(HConstants.CATALOG_FAMILY,
915             HConstants.REGIONINFO_QUALIFIER));
916     final byte[] value = data.getValue(HConstants.CATALOG_FAMILY,
917         HConstants.SERVER_QUALIFIER);
918     if (value != null && value.length > 0) {
919       HServerAddress server = new HServerAddress(Bytes.toString(value));
920       return new Pair<HRegionInfo,HServerAddress>(info, server);
921     } else {
922       //undeployed
923       return new Pair<HRegionInfo, HServerAddress>(info, null);
924     }    
925   }
926 
927   /**
928    * Return the region and current deployment for the region containing
929    * the given row. If the region cannot be found, returns null. If it
930    * is found, but not currently deployed, the second element of the pair
931    * may be null.
932    */
933   Pair<HRegionInfo,HServerAddress> getTableRegionForRow(
934       final byte [] tableName, final byte [] rowKey)
935   throws IOException {
936     final AtomicReference<Pair<HRegionInfo, HServerAddress>> result =
937       new AtomicReference<Pair<HRegionInfo, HServerAddress>>(null);
938     
939     MetaScannerVisitor visitor =
940       new MetaScannerVisitor() {
941         @Override
942         public boolean processRow(Result data) throws IOException {
943           if (data == null || data.size() <= 0)
944             return true;
945           Pair<HRegionInfo, HServerAddress> pair =
946             metaRowToRegionPair(data);
947           if (pair == null) return false;
948           if (!Bytes.equals(pair.getFirst().getTableDesc().getName(),
949                 tableName)) {
950             return false;
951           }
952           result.set(pair);
953           return true;
954         }
955     };
956 
957     MetaScanner.metaScan(conf, visitor, tableName, rowKey, 1);
958     return result.get();
959   }
960   
961   Pair<HRegionInfo,HServerAddress> getTableRegionFromName(
962       final byte [] regionName)
963   throws IOException {
964     byte [] tableName = HRegionInfo.parseRegionName(regionName)[0];
965     
966     Set<MetaRegion> regions = regionManager.getMetaRegionsForTable(tableName);
967     for (MetaRegion m: regions) {
968       byte [] metaRegionName = m.getRegionName();
969       HRegionInterface srvr = connection.getHRegionConnection(m.getServer());
970       Get get = new Get(regionName);
971       get.addColumn(HConstants.CATALOG_FAMILY,
972           HConstants.REGIONINFO_QUALIFIER);
973       get.addColumn(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER);
974       Result data = srvr.get(metaRegionName, get);
975       if(data == null || data.size() <= 0) continue;
976       return metaRowToRegionPair(data);
977     }
978     return null;
979   }
980 
981   /**
982    * Get row from meta table.
983    * @param row
984    * @param family
985    * @return Result
986    * @throws IOException
987    */
988   protected Result getFromMETA(final byte [] row, final byte [] family)
989   throws IOException {
990     MetaRegion meta = this.regionManager.getMetaRegionForRow(row);
991     HRegionInterface srvr = getMETAServer(meta);
992     Get get = new Get(row);
993     get.addFamily(family);
994     return srvr.get(meta.getRegionName(), get);
995   }
996 
997   /*
998    * @param meta
999    * @return Server connection to <code>meta</code> .META. region.
1000    * @throws IOException
1001    */
1002   private HRegionInterface getMETAServer(final MetaRegion meta)
1003   throws IOException {
1004     return this.connection.getHRegionConnection(meta.getServer());
1005   }
1006 
1007   public void modifyTable(final byte[] tableName, HConstants.Modify op,
1008       Writable[] args)
1009   throws IOException {
1010     switch (op) {
1011     case TABLE_SET_HTD:
1012       if (args == null || args.length < 1 ||
1013           !(args[0] instanceof HTableDescriptor))
1014         throw new IOException("SET_HTD request requires an HTableDescriptor");
1015       HTableDescriptor htd = (HTableDescriptor) args[0];
1016       LOG.info("modifyTable(SET_HTD): " + htd);
1017       new ModifyTableMeta(this, tableName, htd).process();
1018       break;
1019 
1020     case TABLE_SPLIT:
1021     case TABLE_COMPACT:
1022     case TABLE_MAJOR_COMPACT:
1023     case TABLE_FLUSH:
1024       if (args != null && args.length > 0) {
1025         if (!(args[0] instanceof ImmutableBytesWritable))
1026           throw new IOException(
1027             "request argument must be ImmutableBytesWritable");
1028         Pair<HRegionInfo,HServerAddress> pair = null;
1029         if(tableName == null) {
1030           byte [] regionName = ((ImmutableBytesWritable)args[0]).get();
1031           pair = getTableRegionFromName(regionName);
1032         } else {
1033           byte [] rowKey = ((ImmutableBytesWritable)args[0]).get();
1034           pair = getTableRegionForRow(tableName, rowKey);
1035         }
1036         LOG.info("About to " + op.toString() + " on " + Bytes.toString(tableName) + " and pair is " + pair);
1037         if (pair != null && pair.getSecond() != null) {
1038           this.regionManager.startAction(pair.getFirst().getRegionName(),
1039             pair.getFirst(), pair.getSecond(), op);
1040         }
1041       } else {
1042         for (Pair<HRegionInfo,HServerAddress> pair: getTableRegions(tableName)) {
1043           if (pair.getSecond() == null) continue; // undeployed
1044           this.regionManager.startAction(pair.getFirst().getRegionName(),
1045             pair.getFirst(), pair.getSecond(), op);
1046         }
1047       }
1048       break;
1049 
1050     case CLOSE_REGION:
1051       if (args == null || args.length < 1 || args.length > 2) {
1052         throw new IOException("Requires at least a region name; " +
1053           "or cannot have more than region name and servername");
1054       }
1055       // Arguments are regionname and an optional server name.
1056       byte [] regionname = ((ImmutableBytesWritable)args[0]).get();
1057       LOG.debug("Attempting to close region: " + Bytes.toStringBinary(regionname));
1058       String hostnameAndPort = null;
1059       if (args.length == 2) {
1060         hostnameAndPort = Bytes.toString(((ImmutableBytesWritable)args[1]).get());
1061       }
1062       // Need hri
1063       Result rr = getFromMETA(regionname, HConstants.CATALOG_FAMILY);
1064       HRegionInfo hri = getHRegionInfo(rr.getRow(), rr);
1065       if (hostnameAndPort == null) {
1066         // Get server from the .META. if it wasn't passed as argument
1067         hostnameAndPort =
1068           Bytes.toString(rr.getValue(HConstants.CATALOG_FAMILY,
1069               HConstants.SERVER_QUALIFIER));
1070       }
1071       // Take region out of the intransistions in case it got stuck there doing
1072       // an open or whatever.
1073       this.regionManager.clearFromInTransition(regionname);
1074       // If hostnameAndPort is still null, then none, exit.
1075       if (hostnameAndPort == null) break;
1076       long startCode =
1077         Bytes.toLong(rr.getValue(HConstants.CATALOG_FAMILY,
1078             HConstants.STARTCODE_QUALIFIER));
1079       String name = HServerInfo.getServerName(hostnameAndPort, startCode);
1080       LOG.info("Marking " + hri.getRegionNameAsString() +
1081         " as closing on " + name + "; cleaning SERVER + STARTCODE; " +
1082           "master will tell regionserver to close region on next heartbeat");
1083       this.regionManager.setClosing(name, hri, hri.isOffline());
1084       MetaRegion meta = this.regionManager.getMetaRegionForRow(regionname);
1085       HRegionInterface srvr = getMETAServer(meta);
1086       HRegion.cleanRegionInMETA(srvr, meta.getRegionName(), hri);
1087       break;
1088 
1089     default:
1090       throw new IOException("unsupported modifyTable op " + op);
1091     }
1092   }
1093 
1094   /**
1095    * @return cluster status
1096    */
1097   public ClusterStatus getClusterStatus() {
1098     ClusterStatus status = new ClusterStatus();
1099     status.setHBaseVersion(VersionInfo.getVersion());
1100     status.setServerInfo(serverManager.getServersToServerInfo().values());
1101     status.setDeadServers(serverManager.getDeadServers());
1102     status.setRegionsInTransition(this.regionManager.getRegionsInTransition());
1103     return status;
1104   }
1105 
1106   // TODO ryan rework this function
1107   /*
1108    * Get HRegionInfo from passed META map of row values.
1109    * Returns null if none found (and logs fact that expected COL_REGIONINFO
1110    * was missing).  Utility method used by scanners of META tables.
1111    * @param row name of the row
1112    * @param map Map to do lookup in.
1113    * @return Null or found HRegionInfo.
1114    * @throws IOException
1115    */
1116   HRegionInfo getHRegionInfo(final byte [] row, final Result res)
1117   throws IOException {
1118     byte[] regioninfo = res.getValue(HConstants.CATALOG_FAMILY,
1119         HConstants.REGIONINFO_QUALIFIER);
1120     if (regioninfo == null) {
1121       StringBuilder sb =  new StringBuilder();
1122       NavigableMap<byte[], byte[]> infoMap =
1123         res.getFamilyMap(HConstants.CATALOG_FAMILY);
1124       for (byte [] e: infoMap.keySet()) {
1125         if (sb.length() > 0) {
1126           sb.append(", ");
1127         }
1128         sb.append(Bytes.toString(HConstants.CATALOG_FAMILY) + ":"
1129             + Bytes.toString(e));
1130       }
1131       LOG.warn(Bytes.toString(HConstants.CATALOG_FAMILY) + ":" +
1132           Bytes.toString(HConstants.REGIONINFO_QUALIFIER)
1133           + " is empty for row: " + Bytes.toString(row) + "; has keys: "
1134           + sb.toString());
1135       return null;
1136     }
1137     return Writables.getHRegionInfo(regioninfo);
1138   }
1139 
1140   /*
1141    * When we find rows in a meta region that has an empty HRegionInfo, we
1142    * clean them up here.
1143    *
1144    * @param s connection to server serving meta region
1145    * @param metaRegionName name of the meta region we scanned
1146    * @param emptyRows the row keys that had empty HRegionInfos
1147    */
1148   protected void deleteEmptyMetaRows(HRegionInterface s,
1149       byte [] metaRegionName,
1150       List<byte []> emptyRows) {
1151     for (byte [] regionName: emptyRows) {
1152       try {
1153         HRegion.removeRegionFromMETA(s, metaRegionName, regionName);
1154         LOG.warn("Removed region: " + Bytes.toString(regionName) +
1155           " from meta region: " +
1156           Bytes.toString(metaRegionName) + " because HRegionInfo was empty");
1157       } catch (IOException e) {
1158         LOG.error("deleting region: " + Bytes.toString(regionName) +
1159             " from meta region: " + Bytes.toString(metaRegionName), e);
1160       }
1161     }
1162   }
1163 
1164   /**
1165    * @see org.apache.zookeeper.Watcher#process(org.apache.zookeeper.WatchedEvent)
1166    */
1167   @Override
1168   public void process(WatchedEvent event) {
1169     LOG.debug("Event " + event.getType() + 
1170               " with state " + event.getState() +  
1171               " with path " + event.getPath());
1172     // Master should kill itself if its session expired or if its
1173     // znode was deleted manually (usually for testing purposes)
1174     if(event.getState() == KeeperState.Expired ||
1175       (event.getType().equals(EventType.NodeDeleted) &&
1176         event.getPath().equals(this.zooKeeperWrapper.getMasterElectionZNode())) &&
1177         !shutdownRequested.get()) {
1178 
1179       LOG.info("Master lost its znode, trying to get a new one");
1180 
1181       // Can we still be the master? If not, goodbye
1182 
1183       zooKeeperWrapper.close();
1184       try {
1185         zooKeeperWrapper =
1186             ZooKeeperWrapper.createInstance(conf, HMaster.class.getName());
1187         zooKeeperWrapper.registerListener(this);
1188         this.zkMasterAddressWatcher.setZookeeper(zooKeeperWrapper);
1189         if(!this.zkMasterAddressWatcher.
1190             writeAddressToZooKeeper(this.address,false)) {
1191           throw new Exception("Another Master is currently active");
1192         }
1193 
1194         // we are a failed over master, reset the fact that we started the 
1195         // cluster
1196         resetClusterStartup();
1197         // Verify the cluster to see if anything happened while we were away
1198         joinCluster();
1199       } catch (Exception e) {
1200         LOG.error("Killing master because of", e);
1201         System.exit(1);
1202       }
1203     }
1204   }
1205 
1206   private static void printUsageAndExit() {
1207     System.err.println("Usage: Master [opts] start|stop");
1208     System.err.println(" start  Start Master. If local mode, start Master and RegionServer in same JVM");
1209     System.err.println(" stop   Start cluster shutdown; Master signals RegionServer shutdown");
1210     System.err.println(" where [opts] are:");
1211     System.err.println("   --minServers=<servers>    Minimum RegionServers needed to host user tables.");
1212     System.err.println("   -D opt=<value>            Override HBase configuration settings.");
1213     System.exit(0);
1214   }
1215 
1216   /**
1217    * Utility for constructing an instance of the passed HMaster class.
1218    * @param masterClass
1219    * @param conf
1220    * @return HMaster instance.
1221    */
1222   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
1223       final Configuration conf)  {
1224     try {
1225       Constructor<? extends HMaster> c =
1226         masterClass.getConstructor(Configuration.class);
1227       return c.newInstance(conf);
1228     } catch (Exception e) {
1229       throw new RuntimeException("Failed construction of " +
1230         "Master: " + masterClass.toString() +
1231         ((e.getCause() != null)? e.getCause().getMessage(): ""), e);
1232     }
1233   }
1234 
1235   /*
1236    * Version of master that will shutdown the passed zk cluster on its way out.
1237    */
1238   static class LocalHMaster extends HMaster {
1239     private MiniZooKeeperCluster zkcluster = null;
1240 
1241     public LocalHMaster(Configuration conf) throws IOException {
1242       super(conf);
1243     }
1244 
1245     @Override
1246     public void run() {
1247       super.run();
1248       if (this.zkcluster != null) {
1249         try {
1250           this.zkcluster.shutdown();
1251         } catch (IOException e) {
1252           e.printStackTrace();
1253         }
1254       }
1255     }
1256 
1257     void setZKCluster(final MiniZooKeeperCluster zkcluster) {
1258       this.zkcluster = zkcluster;
1259     }
1260   }
1261 
1262   protected static void doMain(String [] args,
1263       Class<? extends HMaster> masterClass) {
1264     Configuration conf = HBaseConfiguration.create();
1265 
1266     Options opt = new Options();
1267     opt.addOption("minServers", true, "Minimum RegionServers needed to host user tables");
1268     opt.addOption("D", true, "Override HBase Configuration Settings");
1269     try {
1270       CommandLine cmd = new GnuParser().parse(opt, args);
1271 
1272       if (cmd.hasOption("minServers")) {
1273         String val = cmd.getOptionValue("minServers");
1274         conf.setInt("hbase.regions.server.count.min",
1275             Integer.valueOf(val));
1276         LOG.debug("minServers set to " + val);
1277       }
1278 
1279       if (cmd.hasOption("D")) {
1280         for (String confOpt : cmd.getOptionValues("D")) {
1281           String[] kv = confOpt.split("=", 2);
1282           if (kv.length == 2) {
1283             conf.set(kv[0], kv[1]);
1284             LOG.debug("-D configuration override: " + kv[0] + "=" + kv[1]);
1285           } else {
1286             throw new ParseException("-D option format invalid: " + confOpt);
1287           }
1288         }
1289       }
1290 
1291       if (cmd.getArgList().contains("start")) {
1292         try {
1293           // Print out vm stats before starting up.
1294           RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
1295           if (runtime != null) {
1296             LOG.info("vmName=" + runtime.getVmName() + ", vmVendor=" +
1297               runtime.getVmVendor() + ", vmVersion=" + runtime.getVmVersion());
1298             LOG.info("vmInputArguments=" + runtime.getInputArguments());
1299           }
1300           // If 'local', defer to LocalHBaseCluster instance.  Starts master
1301           // and regionserver both in the one JVM.
1302           if (LocalHBaseCluster.isLocal(conf)) {
1303             final MiniZooKeeperCluster zooKeeperCluster =
1304               new MiniZooKeeperCluster();
1305             File zkDataPath = new File(conf.get("hbase.zookeeper.property.dataDir"));
1306             int zkClientPort = conf.getInt("hbase.zookeeper.property.clientPort", 0);
1307             if (zkClientPort == 0) {
1308               throw new IOException("No config value for hbase.zookeeper.property.clientPort");
1309             }
1310             zooKeeperCluster.setTickTime(conf.getInt("hbase.zookeeper.property.tickTime", 3000));
1311             zooKeeperCluster.setClientPort(zkClientPort);
1312             int clientPort = zooKeeperCluster.startup(zkDataPath);
1313             if (clientPort != zkClientPort) {
1314               String errorMsg = "Couldnt start ZK at requested address of " +
1315                   zkClientPort + ", instead got: " + clientPort + ". Aborting. Why? " +
1316                   "Because clients (eg shell) wont be able to find this ZK quorum";
1317               System.err.println(errorMsg);
1318               throw new IOException(errorMsg);
1319             }
1320             conf.set("hbase.zookeeper.property.clientPort",
1321               Integer.toString(clientPort));
1322             // Need to have the zk cluster shutdown when master is shutdown.
1323             // Run a subclass that does the zk cluster shutdown on its way out.
1324             LocalHBaseCluster cluster = new LocalHBaseCluster(conf, 1,
1325               LocalHMaster.class, HRegionServer.class);
1326             ((LocalHMaster)cluster.getMaster()).setZKCluster(zooKeeperCluster);
1327             cluster.startup();
1328           } else {
1329             HMaster master = constructMaster(masterClass, conf);
1330             if (master.shutdownRequested.get()) {
1331               LOG.info("Won't bring the Master up as a shutdown is requested");
1332               return;
1333             }
1334             master.start();
1335           }
1336         } catch (Throwable t) {
1337           LOG.error("Failed to start master", t);
1338           System.exit(-1);
1339         }
1340       } else if (cmd.getArgList().contains("stop")) {
1341         HBaseAdmin adm = null;
1342         try {
1343           adm = new HBaseAdmin(conf);
1344         } catch (MasterNotRunningException e) {
1345           LOG.error("Master not running");
1346           System.exit(0);
1347         }
1348         try {
1349           adm.shutdown();
1350         } catch (Throwable t) {
1351           LOG.error("Failed to stop master", t);
1352           System.exit(-1);
1353         }
1354       } else {
1355         throw new ParseException("Unknown argument(s): " +
1356             org.apache.commons.lang.StringUtils.join(cmd.getArgs(), " "));
1357       }
1358     } catch (ParseException e) {
1359       LOG.error("Could not parse: ", e);
1360       printUsageAndExit();
1361     }
1362   }
1363 
1364   public Map<String, Integer> getTableFragmentation() throws IOException {
1365     long now = System.currentTimeMillis();
1366     // only check every two minutes by default
1367     int check = this.conf.getInt("hbase.master.fragmentation.check.frequency", 2 * 60 * 1000);
1368     if (lastFragmentationQuery == -1 || now - lastFragmentationQuery > check) {
1369       fragmentation = FSUtils.getTableFragmentation(this);
1370       lastFragmentationQuery = now;
1371     }
1372     return fragmentation;
1373   }
1374 
1375   /**
1376    * Main program
1377    * @param args
1378    */
1379   public static void main(String [] args) {
1380     doMain(args, HMaster.class);
1381   }
1382 }