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