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