View Javadoc

1   /**
2    * Copyright 2011 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.IOException;
23  import java.lang.reflect.Constructor;
24  import java.lang.reflect.InvocationTargetException;
25  import java.net.InetAddress;
26  import java.net.InetSocketAddress;
27  import java.util.ArrayList;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.concurrent.atomic.AtomicReference;
31  import java.util.Set;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.Chore;
37  import org.apache.hadoop.hbase.ClusterStatus;
38  import org.apache.hadoop.hbase.HColumnDescriptor;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.HRegionInfo;
41  import org.apache.hadoop.hbase.HServerLoad;
42  import org.apache.hadoop.hbase.HTableDescriptor;
43  import org.apache.hadoop.hbase.MasterNotRunningException;
44  import org.apache.hadoop.hbase.Server;
45  import org.apache.hadoop.hbase.ServerName;
46  import org.apache.hadoop.hbase.TableDescriptors;
47  import org.apache.hadoop.hbase.TableNotDisabledException;
48  import org.apache.hadoop.hbase.TableNotFoundException;
49  import org.apache.hadoop.hbase.UnknownRegionException;
50  import org.apache.hadoop.hbase.catalog.CatalogTracker;
51  import org.apache.hadoop.hbase.catalog.MetaReader;
52  import org.apache.hadoop.hbase.client.HConnectionManager;
53  import org.apache.hadoop.hbase.client.MetaScanner;
54  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
55  import org.apache.hadoop.hbase.client.Result;
56  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
57  import org.apache.hadoop.hbase.executor.ExecutorService;
58  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
59  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
60  import org.apache.hadoop.hbase.ipc.HBaseRPC;
61  import org.apache.hadoop.hbase.ipc.HBaseServer;
62  import org.apache.hadoop.hbase.ipc.HMasterInterface;
63  import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
64  import org.apache.hadoop.hbase.ipc.ProtocolSignature;
65  import org.apache.hadoop.hbase.ipc.RpcServer;
66  import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
67  import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
68  import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
69  import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
70  import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
71  import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
72  import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
73  import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
74  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
75  import org.apache.hadoop.hbase.master.RegionPlan;
76  import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
77  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
78  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
79  import org.apache.hadoop.hbase.replication.regionserver.Replication;
80  import org.apache.hadoop.hbase.security.User;
81  import org.apache.hadoop.hbase.util.Bytes;
82  import org.apache.hadoop.hbase.util.FSTableDescriptors;
83  import org.apache.hadoop.hbase.util.HasThread;
84  import org.apache.hadoop.hbase.util.InfoServer;
85  import org.apache.hadoop.hbase.util.Pair;
86  import org.apache.hadoop.hbase.util.Sleeper;
87  import org.apache.hadoop.hbase.util.Threads;
88  import org.apache.hadoop.hbase.util.VersionInfo;
89  import org.apache.hadoop.hbase.zookeeper.ClusterId;
90  import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
91  import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
92  import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
93  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
94  import org.apache.hadoop.io.MapWritable;
95  import org.apache.hadoop.io.Text;
96  import org.apache.hadoop.net.DNS;
97  import org.apache.zookeeper.KeeperException;
98  import org.apache.zookeeper.Watcher;
99  
100 /**
101  * HMaster is the "master server" for HBase. An HBase cluster has one active
102  * master.  If many masters are started, all compete.  Whichever wins goes on to
103  * run the cluster.  All others park themselves in their constructor until
104  * master or cluster shutdown or until the active master loses its lease in
105  * zookeeper.  Thereafter, all running master jostle to take over master role.
106  *
107  * <p>The Master can be asked shutdown the cluster. See {@link #shutdown()}.  In
108  * this case it will tell all regionservers to go down and then wait on them
109  * all reporting in that they are down.  This master will then shut itself down.
110  *
111  * <p>You can also shutdown just this master.  Call {@link #stopMaster()}.
112  *
113  * @see HMasterInterface
114  * @see HMasterRegionInterface
115  * @see Watcher
116  */
117 public class HMaster extends HasThread
118 implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
119   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
120 
121   // MASTER is name of the webapp and the attribute name used stuffing this
122   //instance into web context.
123   public static final String MASTER = "master";
124 
125   // The configuration for the Master
126   private final Configuration conf;
127   // server for the web ui
128   private InfoServer infoServer;
129 
130   // Our zk client.
131   private ZooKeeperWatcher zooKeeper;
132   // Manager and zk listener for master election
133   private ActiveMasterManager activeMasterManager;
134   // Region server tracker
135   private RegionServerTracker regionServerTracker;
136   // Draining region server tracker
137   private DrainingServerTracker drainingServerTracker;
138 
139   // RPC server for the HMaster
140   private final RpcServer rpcServer;
141 
142   /**
143    * This servers address.
144    */
145   private final InetSocketAddress isa;
146 
147   // Metrics for the HMaster
148   private final MasterMetrics metrics;
149   // file system manager for the master FS operations
150   private MasterFileSystem fileSystemManager;
151 
152   // server manager to deal with region server info
153   private ServerManager serverManager;
154 
155   // manager of assignment nodes in zookeeper
156   AssignmentManager assignmentManager;
157   // manager of catalog regions
158   private CatalogTracker catalogTracker;
159   // Cluster status zk tracker and local setter
160   private ClusterStatusTracker clusterStatusTracker;
161   
162   // buffer for "fatal error" notices from region servers
163   // in the cluster. This is only used for assisting
164   // operations/debugging.
165   private MemoryBoundedLogMessageBuffer rsFatals;
166 
167   // This flag is for stopping this Master instance.  Its set when we are
168   // stopping or aborting
169   private volatile boolean stopped = false;
170   // Set on abort -- usually failure of our zk session.
171   private volatile boolean abort = false;
172   // flag set after we become the active master (used for testing)
173   private volatile boolean isActiveMaster = false;
174   // flag set after we complete initialization once active (used for testing)
175   private volatile boolean initialized = false;
176 
177   // Instance of the hbase executor service.
178   ExecutorService executorService;
179 
180   private LoadBalancer balancer;
181   private Thread balancerChore;
182   // If 'true', the balancer is 'on'.  If 'false', the balancer will not run.
183   private volatile boolean balanceSwitch = true;
184 
185   private CatalogJanitor catalogJanitorChore;
186   private LogCleaner logCleaner;
187 
188   private MasterCoprocessorHost cpHost;
189   private final ServerName serverName;
190 
191   private TableDescriptors tableDescriptors;
192   
193   /**
194    * Initializes the HMaster. The steps are as follows:
195    * <p>
196    * <ol>
197    * <li>Initialize HMaster RPC and address
198    * <li>Connect to ZooKeeper.
199    * </ol>
200    * <p>
201    * Remaining steps of initialization occur in {@link #run()} so that they
202    * run in their own thread rather than within the context of the constructor.
203    * @throws InterruptedException
204    */
205   public HMaster(final Configuration conf)
206   throws IOException, KeeperException, InterruptedException {
207     this.conf = new Configuration(conf);
208     // Disable the block cache on the master
209     this.conf.setFloat(CacheConfig.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
210     // Set how many times to retry talking to another server over HConnection.
211     HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
212     // Server to handle client requests.
213     String hostname = DNS.getDefaultHost(
214       conf.get("hbase.master.dns.interface", "default"),
215       conf.get("hbase.master.dns.nameserver", "default"));
216     int port = conf.getInt(HConstants.MASTER_PORT, HConstants.DEFAULT_MASTER_PORT);
217     // Creation of a HSA will force a resolve.
218     InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
219     if (initialIsa.getAddress() == null) {
220       throw new IllegalArgumentException("Failed resolve of " + this.isa);
221     }
222     int numHandlers = conf.getInt("hbase.master.handler.count",
223       conf.getInt("hbase.regionserver.handler.count", 25));
224     this.rpcServer = HBaseRPC.getServer(this,
225       new Class<?>[]{HMasterInterface.class, HMasterRegionInterface.class},
226         initialIsa.getHostName(), // BindAddress is IP we got for this server.
227         initialIsa.getPort(),
228         numHandlers,
229         0, // we dont use high priority handlers in master
230         conf.getBoolean("hbase.rpc.verbose", false), conf,
231         0); // this is a DNC w/o high priority handlers
232     // Set our address.
233     this.isa = this.rpcServer.getListenerAddress();
234     this.serverName = new ServerName(this.isa.getHostName(),
235       this.isa.getPort(), System.currentTimeMillis());
236     this.rsFatals = new MemoryBoundedLogMessageBuffer(
237         conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
238 
239     // initialize server principal (if using secure Hadoop)
240     User.login(conf, "hbase.master.keytab.file",
241       "hbase.master.kerberos.principal", this.isa.getHostName());
242 
243     // set the thread name now we have an address
244     setName(MASTER + "-" + this.serverName.toString());
245 
246     Replication.decorateMasterConfiguration(this.conf);
247 
248     // Hack! Maps DFSClient => Master for logs.  HDFS made this
249     // config param for task trackers, but we can piggyback off of it.
250     if (this.conf.get("mapred.task.id") == null) {
251       this.conf.set("mapred.task.id", "hb_m_" + this.serverName.toString());
252     }
253 
254     this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":" + isa.getPort(), this, true);
255     this.rpcServer.startThreads();
256     this.metrics = new MasterMetrics(getServerName().toString());
257   }
258 
259   /**
260    * Stall startup if we are designated a backup master; i.e. we want someone
261    * else to become the master before proceeding.
262    * @param c
263    * @param amm
264    * @throws InterruptedException
265    */
266   private static void stallIfBackupMaster(final Configuration c,
267       final ActiveMasterManager amm)
268   throws InterruptedException {
269     // If we're a backup master, stall until a primary to writes his address
270     if (!c.getBoolean(HConstants.MASTER_TYPE_BACKUP,
271       HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
272       return;
273     }
274     LOG.debug("HMaster started in backup mode.  " +
275       "Stalling until master znode is written.");
276     // This will only be a minute or so while the cluster starts up,
277     // so don't worry about setting watches on the parent znode
278     while (!amm.isActiveMaster()) {
279       LOG.debug("Waiting for master address ZNode to be written " +
280         "(Also watching cluster state node)");
281       Thread.sleep(c.getInt("zookeeper.session.timeout", 180 * 1000));
282     }
283   }
284 
285   /**
286    * Main processing loop for the HMaster.
287    * <ol>
288    * <li>Block until becoming active master
289    * <li>Finish initialization via finishInitialization(MonitoredTask)
290    * <li>Enter loop until we are stopped
291    * <li>Stop services and perform cleanup once stopped
292    * </ol>
293    */
294   @Override
295   public void run() {
296     MonitoredTask startupStatus =
297       TaskMonitor.get().createStatus("Master startup");
298     startupStatus.setDescription("Master startup");
299     try {
300       /*
301        * Block on becoming the active master.
302        *
303        * We race with other masters to write our address into ZooKeeper.  If we
304        * succeed, we are the primary/active master and finish initialization.
305        *
306        * If we do not succeed, there is another active master and we should
307        * now wait until it dies to try and become the next active master.  If we
308        * do not succeed on our first attempt, this is no longer a cluster startup.
309        */
310       becomeActiveMaster(startupStatus);
311 
312       // We are either the active master or we were asked to shutdown
313       if (!this.stopped) {
314         finishInitialization(startupStatus);
315         loop();
316       }
317     } catch (Throwable t) {
318       abort("Unhandled exception. Starting shutdown.", t);
319     } finally {
320       startupStatus.cleanup();
321       
322       stopChores();
323       // Wait for all the remaining region servers to report in IFF we were
324       // running a cluster shutdown AND we were NOT aborting.
325       if (!this.abort && this.serverManager != null &&
326           this.serverManager.isClusterShutdown()) {
327         this.serverManager.letRegionServersShutdown();
328       }
329       stopServiceThreads();
330       // Stop services started for both backup and active masters
331       if (this.activeMasterManager != null) this.activeMasterManager.stop();
332       if (this.catalogTracker != null) this.catalogTracker.stop();
333       if (this.serverManager != null) this.serverManager.stop();
334       if (this.assignmentManager != null) this.assignmentManager.stop();
335       if (this.fileSystemManager != null) this.fileSystemManager.stop();
336       this.zooKeeper.close();
337     }
338     LOG.info("HMaster main thread exiting");
339   }
340 
341   /**
342    * Try becoming active master.
343    * @param startupStatus 
344    * @return True if we could successfully become the active master.
345    * @throws InterruptedException
346    */
347   private boolean becomeActiveMaster(MonitoredTask startupStatus)
348   throws InterruptedException {
349     // TODO: This is wrong!!!! Should have new servername if we restart ourselves,
350     // if we come back to life.
351     this.activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName,
352         this);
353     this.zooKeeper.registerListener(activeMasterManager);
354     stallIfBackupMaster(this.conf, this.activeMasterManager);
355     return this.activeMasterManager.blockUntilBecomingActiveMaster(startupStatus);
356   }
357 
358   /**
359    * Initialize all ZK based system trackers.
360    * @throws IOException
361    * @throws InterruptedException
362    */
363   private void initializeZKBasedSystemTrackers() throws IOException,
364       InterruptedException, KeeperException {
365     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf,
366         this, conf.getInt("hbase.master.catalog.timeout", Integer.MAX_VALUE));
367     this.catalogTracker.start();
368 
369     this.assignmentManager = new AssignmentManager(this, serverManager,
370         this.catalogTracker, this.executorService);
371     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
372     zooKeeper.registerListenerFirst(assignmentManager);
373 
374     this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
375         this.serverManager);
376     this.regionServerTracker.start();
377 
378     this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this,
379       this.serverManager);
380     this.drainingServerTracker.start();
381 
382     // Set the cluster as up.  If new RSs, they'll be waiting on this before
383     // going ahead with their startup.
384     this.clusterStatusTracker = new ClusterStatusTracker(getZooKeeper(), this);
385     this.clusterStatusTracker.start();
386     boolean wasUp = this.clusterStatusTracker.isClusterUp();
387     if (!wasUp) this.clusterStatusTracker.setClusterUp();
388 
389     LOG.info("Server active/primary master; " + this.serverName +
390         ", sessionid=0x" +
391         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
392         ", cluster-up flag was=" + wasUp);
393   }
394 
395   private void loop() {
396     // Check if we should stop every second.
397     Sleeper sleeper = new Sleeper(1000, this);
398     while (!this.stopped) {
399       sleeper.sleep();
400     }
401   }
402 
403   /**
404    * Finish initialization of HMaster after becoming the primary master.
405    *
406    * <ol>
407    * <li>Initialize master components - file system manager, server manager,
408    *     assignment manager, region server tracker, catalog tracker, etc</li>
409    * <li>Start necessary service threads - rpc server, info server,
410    *     executor services, etc</li>
411    * <li>Set cluster as UP in ZooKeeper</li>
412    * <li>Wait for RegionServers to check-in</li>
413    * <li>Split logs and perform data recovery, if necessary</li>
414    * <li>Ensure assignment of root and meta regions<li>
415    * <li>Handle either fresh cluster start or master failover</li>
416    * </ol>
417    *
418    * @throws IOException
419    * @throws InterruptedException
420    * @throws KeeperException
421    */
422   private void finishInitialization(MonitoredTask status)
423   throws IOException, InterruptedException, KeeperException {
424 
425     isActiveMaster = true;
426 
427     /*
428      * We are active master now... go initialize components we need to run.
429      * Note, there may be dross in zk from previous runs; it'll get addressed
430      * below after we determine if cluster startup or failover.
431      */
432 
433     status.setStatus("Initializing Master file system");
434     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
435     this.fileSystemManager = new MasterFileSystem(this, this, metrics);
436 
437     this.tableDescriptors =
438       new FSTableDescriptors(this.fileSystemManager.getFileSystem(),
439       this.fileSystemManager.getRootDir());
440 
441     // publish cluster ID
442     status.setStatus("Publishing Cluster ID in ZooKeeper");
443     ClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
444 
445     this.executorService = new ExecutorService(getServerName().toString());
446 
447     this.serverManager = new ServerManager(this, this);
448 
449     status.setStatus("Initializing ZK system trackers");
450     initializeZKBasedSystemTrackers();
451 
452     // initialize master side coprocessors before we start handling requests
453     status.setStatus("Initializing master coprocessors");
454     this.cpHost = new MasterCoprocessorHost(this, this.conf);
455 
456     // start up all service threads.
457     status.setStatus("Initializing master service threads");
458     startServiceThreads();
459 
460     // Wait for region servers to report in.
461     this.serverManager.waitForRegionServers(status);
462     // Check zk for regionservers that are up but didn't register
463     for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
464       if (!this.serverManager.isServerOnline(sn)) {
465         // Not registered; add it.
466         LOG.info("Registering server found up in zk but who has not yet " +
467           "reported in: " + sn);
468         this.serverManager.recordNewServer(sn, HServerLoad.EMPTY_HSERVERLOAD);
469       }
470     }
471 
472     // TODO: Should do this in background rather than block master startup
473     status.setStatus("Splitting logs after master startup");
474     this.fileSystemManager.
475       splitLogAfterStartup(this.serverManager.getOnlineServers().keySet());
476 
477     // Make sure root and meta assigned before proceeding.
478     assignRootAndMeta(status);
479     // Update meta with new HRI if required. i.e migrate all HRI with HTD to
480     // HRI with out HTD in meta and update the status in ROOT. This must happen
481     // before we assign all user regions or else the assignment will fail.
482     // TODO: Remove this when we do 0.94.
483     org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD.
484       updateMetaWithNewHRI(this);
485 
486     // Fixup assignment manager status
487     status.setStatus("Starting assignment manager");
488     this.assignmentManager.joinCluster();
489 
490     this.balancer.setClusterStatus(getClusterStatus());
491     this.balancer.setMasterServices(this);
492     
493     // Start balancer and meta catalog janitor after meta and regions have
494     // been assigned.
495     status.setStatus("Starting balancer and catalog janitor");
496     this.balancerChore = getAndStartBalancerChore(this);
497     this.catalogJanitorChore = new CatalogJanitor(this, this);
498     Threads.setDaemonThreadRunning(catalogJanitorChore.getThread());
499 
500     status.markComplete("Initialization successful");
501     LOG.info("Master has completed initialization");
502     initialized = true;
503 
504     if (this.cpHost != null) {
505       // don't let cp initialization errors kill the master
506       try {
507         this.cpHost.postStartMaster();
508       } catch (IOException ioe) {
509         LOG.error("Coprocessor postStartMaster() hook failed", ioe);
510       }
511     }
512   }
513 
514   /**
515    * Check <code>-ROOT-</code> and <code>.META.</code> are assigned.  If not,
516    * assign them.
517    * @throws InterruptedException
518    * @throws IOException
519    * @throws KeeperException
520    * @return Count of regions we assigned.
521    */
522   int assignRootAndMeta(MonitoredTask status)
523   throws InterruptedException, IOException, KeeperException {
524     int assigned = 0;
525     long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
526 
527     // Work on ROOT region.  Is it in zk in transition?
528     status.setStatus("Assigning ROOT region");
529     boolean rit = this.assignmentManager.
530       processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.ROOT_REGIONINFO);
531     ServerName expiredServer = null;
532     if (!catalogTracker.verifyRootRegionLocation(timeout)) {
533       ServerName currentRootServer = this.catalogTracker.getRootLocation();
534       if (expireIfOnline(currentRootServer)) {
535         // We are expiring this server. The processing of expiration will assign
536         // root so don't do it here.
537         expiredServer = currentRootServer;
538       } else {
539         // Root was not on an online server when we failed verification
540         this.assignmentManager.assignRoot();
541       }
542       this.catalogTracker.waitForRoot();
543       //This guarantees that the transition has completed
544       this.assignmentManager.waitForAssignment(HRegionInfo.ROOT_REGIONINFO);
545       assigned++;
546     } else {
547       // Region already assigned.  We didn't assign it.  Add to in-memory state.
548       this.assignmentManager.regionOnline(HRegionInfo.ROOT_REGIONINFO,
549         this.catalogTracker.getRootLocation());
550     }
551     LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit +
552       ", location=" + catalogTracker.getRootLocation());
553 
554     // Work on meta region
555     status.setStatus("Assigning META region");
556     rit = this.assignmentManager.
557       processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
558     if (!this.catalogTracker.verifyMetaRegionLocation(timeout)) {
559       ServerName currentMetaServer =
560         this.catalogTracker.getMetaLocationOrReadLocationFromRoot();
561       if (currentMetaServer != null && currentMetaServer.equals(expiredServer)) {
562         // We are expiring the server that is carrying meta already.
563         // The expiration processing will take care of reassigning meta.
564         expireIfOnline(currentMetaServer);
565       } else {
566         this.assignmentManager.assignMeta();
567       }
568       this.catalogTracker.waitForMeta();
569       // Above check waits for general meta availability but this does not
570       // guarantee that the transition has completed
571       this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
572       assigned++;
573     } else {
574       // Region already assigned.  We didnt' assign it.  Add to in-memory state.
575       this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
576         this.catalogTracker.getMetaLocation());
577     }
578     LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
579       ", location=" + catalogTracker.getMetaLocation());
580     status.setStatus("META and ROOT assigned.");
581     return assigned;
582   }
583 
584   /**
585    * Expire a server if we find it is one of the online servers set.
586    * @param sn ServerName to check.
587    * @return True if server was online and so we expired it as unreachable.
588    */
589   private boolean expireIfOnline(final ServerName sn) {
590     if (sn == null) return false;
591     if (!this.serverManager.isServerOnline(sn)) return false;
592     LOG.info("Forcing expiration of " + sn);
593     this.serverManager.expireServer(sn);
594     return true;
595   }
596 
597   @Override
598   public ProtocolSignature getProtocolSignature(
599       String protocol, long version, int clientMethodsHashCode)
600   throws IOException {
601     if (HMasterInterface.class.getName().equals(protocol)) {
602       return new ProtocolSignature(HMasterInterface.VERSION, null);
603     } else if (HMasterRegionInterface.class.getName().equals(protocol)) {
604       return new ProtocolSignature(HMasterRegionInterface.VERSION, null);
605     }
606     throw new IOException("Unknown protocol: " + protocol);
607   }
608 
609   public long getProtocolVersion(String protocol, long clientVersion) {
610     if (HMasterInterface.class.getName().equals(protocol)) {
611       return HMasterInterface.VERSION;
612     } else if (HMasterRegionInterface.class.getName().equals(protocol)) {
613       return HMasterRegionInterface.VERSION;
614     }
615     // unknown protocol
616     LOG.warn("Version requested for unimplemented protocol: "+protocol);
617     return -1;
618   }
619 
620   @Override
621   public TableDescriptors getTableDescriptors() {
622     return this.tableDescriptors;
623   }
624 
625   /** @return InfoServer object. Maybe null.*/
626   public InfoServer getInfoServer() {
627     return this.infoServer;
628   }
629 
630   @Override
631   public Configuration getConfiguration() {
632     return this.conf;
633   }
634 
635   @Override
636   public ServerManager getServerManager() {
637     return this.serverManager;
638   }
639 
640   @Override
641   public ExecutorService getExecutorService() {
642     return this.executorService;
643   }
644 
645   @Override
646   public MasterFileSystem getMasterFileSystem() {
647     return this.fileSystemManager;
648   }
649 
650   /**
651    * Get the ZK wrapper object - needed by master_jsp.java
652    * @return the zookeeper wrapper
653    */
654   public ZooKeeperWatcher getZooKeeperWatcher() {
655     return this.zooKeeper;
656   }
657 
658   /*
659    * Start up all services. If any of these threads gets an unhandled exception
660    * then they just die with a logged message.  This should be fine because
661    * in general, we do not expect the master to get such unhandled exceptions
662    *  as OOMEs; it should be lightly loaded. See what HRegionServer does if
663    *  need to install an unexpected exception handler.
664    */
665   private void startServiceThreads() throws IOException{
666  
667    // Start the executor service pools
668    this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
669       conf.getInt("hbase.master.executor.openregion.threads", 5));
670    this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
671       conf.getInt("hbase.master.executor.closeregion.threads", 5));
672    this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
673       conf.getInt("hbase.master.executor.serverops.threads", 3));
674    this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
675       conf.getInt("hbase.master.executor.serverops.threads", 5));
676    
677    // We depend on there being only one instance of this executor running
678    // at a time.  To do concurrency, would need fencing of enable/disable of
679    // tables.
680    this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
681 
682    // Start log cleaner thread
683    String n = Thread.currentThread().getName();
684    this.logCleaner =
685       new LogCleaner(conf.getInt("hbase.master.cleaner.interval", 60 * 1000),
686          this, conf, getMasterFileSystem().getFileSystem(),
687          getMasterFileSystem().getOldLogDir());
688          Threads.setDaemonThreadRunning(logCleaner.getThread(), n + ".oldLogCleaner");
689 
690    // Put up info server.
691    int port = this.conf.getInt("hbase.master.info.port", 60010);
692    if (port >= 0) {
693      String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
694      this.infoServer = new InfoServer(MASTER, a, port, false, this.conf);
695      this.infoServer.addServlet("status", "/master-status", MasterStatusServlet.class);
696      this.infoServer.addServlet("dump", "/dump", MasterDumpServlet.class);
697      this.infoServer.setAttribute(MASTER, this);
698      this.infoServer.start();
699     }
700    
701     // Start allowing requests to happen.
702     this.rpcServer.openServer();
703     if (LOG.isDebugEnabled()) {
704       LOG.debug("Started service threads");
705     }
706 
707   }
708 
709   private void stopServiceThreads() {
710     if (LOG.isDebugEnabled()) {
711       LOG.debug("Stopping service threads");
712     }
713     if (this.rpcServer != null) this.rpcServer.stop();
714     // Clean up and close up shop
715     if (this.logCleaner!= null) this.logCleaner.interrupt();
716     if (this.infoServer != null) {
717       LOG.info("Stopping infoServer");
718       try {
719         this.infoServer.stop();
720       } catch (Exception ex) {
721         ex.printStackTrace();
722       }
723     }
724     if (this.executorService != null) this.executorService.shutdown();
725   }
726 
727   private static Thread getAndStartBalancerChore(final HMaster master) {
728     String name = master.getServerName() + "-BalancerChore";
729     int balancerPeriod =
730       master.getConfiguration().getInt("hbase.balancer.period", 300000);
731     // Start up the load balancer chore
732     Chore chore = new Chore(name, balancerPeriod, master) {
733       @Override
734       protected void chore() {
735         master.balance();
736       }
737     };
738     return Threads.setDaemonThreadRunning(chore.getThread());
739   }
740 
741   private void stopChores() {
742     if (this.balancerChore != null) {
743       this.balancerChore.interrupt();
744     }
745     if (this.catalogJanitorChore != null) {
746       this.catalogJanitorChore.interrupt();
747     }
748   }
749 
750   @Override
751   public MapWritable regionServerStartup(final int port,
752     final long serverStartCode, final long serverCurrentTime)
753   throws IOException {
754     // Register with server manager
755     InetAddress ia = HBaseServer.getRemoteIp();
756     ServerName rs = this.serverManager.regionServerStartup(ia, port,
757       serverStartCode, serverCurrentTime);
758     // Send back some config info
759     MapWritable mw = createConfigurationSubset();
760     mw.put(new Text(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER),
761       new Text(rs.getHostname()));
762     return mw;
763   }
764 
765   /**
766    * @return Subset of configuration to pass initializing regionservers: e.g.
767    * the filesystem to use and root directory to use.
768    */
769   protected MapWritable createConfigurationSubset() {
770     MapWritable mw = addConfig(new MapWritable(), HConstants.HBASE_DIR);
771     return addConfig(mw, "fs.default.name");
772   }
773 
774   private MapWritable addConfig(final MapWritable mw, final String key) {
775     mw.put(new Text(key), new Text(this.conf.get(key)));
776     return mw;
777   }
778 
779   @Override
780   public void regionServerReport(final byte [] sn, final HServerLoad hsl)
781   throws IOException {
782     this.serverManager.regionServerReport(ServerName.parseVersionedServerName(sn), hsl);
783     if (hsl != null && this.metrics != null) {
784       // Up our metrics.
785       this.metrics.incrementRequests(hsl.getTotalNumberOfRequests());
786     }
787   }
788 
789   @Override
790   public void reportRSFatalError(byte [] sn, String errorText) {
791     String msg = "Region server " + Bytes.toString(sn) +
792       " reported a fatal error:\n" + errorText;
793     LOG.error(msg);
794     rsFatals.add(msg);
795   }
796 
797   public boolean isMasterRunning() {
798     return !isStopped();
799   }
800 
801   /**
802    * @return Maximum time we should run balancer for
803    */
804   private int getBalancerCutoffTime() {
805     int balancerCutoffTime =
806       getConfiguration().getInt("hbase.balancer.max.balancing", -1);
807     if (balancerCutoffTime == -1) {
808       // No time period set so create one -- do half of balancer period.
809       int balancerPeriod =
810         getConfiguration().getInt("hbase.balancer.period", 300000);
811       balancerCutoffTime = balancerPeriod / 2;
812       // If nonsense period, set it to balancerPeriod
813       if (balancerCutoffTime <= 0) balancerCutoffTime = balancerPeriod;
814     }
815     return balancerCutoffTime;
816   }
817 
818   @Override
819   public boolean balance() {
820     // If balance not true, don't run balancer.
821     if (!this.balanceSwitch) return false;
822     // Do this call outside of synchronized block.
823     int maximumBalanceTime = getBalancerCutoffTime();
824     long cutoffTime = System.currentTimeMillis() + maximumBalanceTime;
825     boolean balancerRan;
826     synchronized (this.balancer) {
827       // Only allow one balance run at at time.
828       if (this.assignmentManager.isRegionsInTransition()) {
829         LOG.debug("Not running balancer because " +
830           this.assignmentManager.getRegionsInTransition().size() +
831           " region(s) in transition: " +
832           org.apache.commons.lang.StringUtils.
833             abbreviate(this.assignmentManager.getRegionsInTransition().toString(), 256));
834         return false;
835       }
836       if (this.serverManager.areDeadServersInProgress()) {
837         LOG.debug("Not running balancer because processing dead regionserver(s): " +
838           this.serverManager.getDeadServers());
839         return false;
840       }
841 
842       if (this.cpHost != null) {
843         try {
844           if (this.cpHost.preBalance()) {
845             LOG.debug("Coprocessor bypassing balancer request");
846             return false;
847           }
848         } catch (IOException ioe) {
849           LOG.error("Error invoking master coprocessor preBalance()", ioe);
850           return false;
851         }
852       }
853 
854       Map<ServerName, List<HRegionInfo>> assignments =
855         this.assignmentManager.getAssignments();
856       // Returned Map from AM does not include mention of servers w/o assignments.
857       for (Map.Entry<ServerName, HServerLoad> e:
858           this.serverManager.getOnlineServers().entrySet()) {
859         if (!assignments.containsKey(e.getKey())) {
860           assignments.put(e.getKey(), new ArrayList<HRegionInfo>());
861         }
862       }
863       List<RegionPlan> plans = this.balancer.balanceCluster(assignments);
864       int rpCount = 0;	// number of RegionPlans balanced so far
865       long totalRegPlanExecTime = 0;
866       balancerRan = plans != null;
867       if (plans != null && !plans.isEmpty()) {
868         for (RegionPlan plan: plans) {
869           LOG.info("balance " + plan);
870           long balStartTime = System.currentTimeMillis();
871           this.assignmentManager.balance(plan);
872           totalRegPlanExecTime += System.currentTimeMillis()-balStartTime;
873           rpCount++;
874           if (rpCount < plans.size() &&
875               // if performing next balance exceeds cutoff time, exit the loop
876               (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
877             LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
878               maximumBalanceTime);
879             break;
880           }
881         }
882       }
883       if (this.cpHost != null) {
884         try {
885           this.cpHost.postBalance();
886         } catch (IOException ioe) {
887           // balancing already succeeded so don't change the result
888           LOG.error("Error invoking master coprocessor postBalance()", ioe);
889         }
890       }
891     }
892     return balancerRan;
893   }
894 
895   @Override
896   public boolean balanceSwitch(final boolean b) {
897     boolean oldValue = this.balanceSwitch;
898     boolean newValue = b;
899     try {
900       if (this.cpHost != null) {
901         newValue = this.cpHost.preBalanceSwitch(newValue);
902       }
903       this.balanceSwitch = newValue;
904       LOG.info("Balance=" + newValue);
905       if (this.cpHost != null) {
906         this.cpHost.postBalanceSwitch(oldValue, newValue);
907       }
908     } catch (IOException ioe) {
909       LOG.warn("Error flipping balance switch", ioe);
910     }
911     return oldValue;
912   }
913 
914   /**
915    * Switch for the background CatalogJanitor thread.
916    * Used for testing.  The thread will continue to run.  It will just be a noop
917    * if disabled.
918    * @param b If false, the catalog janitor won't do anything.
919    */
920   public void setCatalogJanitorEnabled(final boolean b) {
921     ((CatalogJanitor)this.catalogJanitorChore).setEnabled(b);
922   }
923 
924   @Override
925   public void move(final byte[] encodedRegionName, final byte[] destServerName)
926   throws UnknownRegionException {
927     Pair<HRegionInfo, ServerName> p =
928       this.assignmentManager.getAssignment(encodedRegionName);
929     if (p == null)
930       throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
931     HRegionInfo hri = p.getFirst();
932     ServerName dest = null;
933     if (destServerName == null || destServerName.length == 0) {
934       LOG.info("Passed destination servername is null/empty so " +
935         "choosing a server at random");
936       this.assignmentManager.clearRegionPlan(hri);
937       // Unassign will reassign it elsewhere choosing random server.
938       this.assignmentManager.unassign(hri);
939     } else {
940       dest = new ServerName(Bytes.toString(destServerName));
941       try {
942         if (this.cpHost != null) {
943           if (this.cpHost.preMove(p.getFirst(), p.getSecond(), dest)) {
944             return;
945           }
946         }
947         RegionPlan rp = new RegionPlan(p.getFirst(), p.getSecond(), dest);
948         LOG.info("Added move plan " + rp + ", running balancer");
949         this.assignmentManager.balance(rp);
950         if (this.cpHost != null) {
951           this.cpHost.postMove(p.getFirst(), p.getSecond(), dest);
952         }
953       } catch (IOException ioe) {
954         UnknownRegionException ure = new UnknownRegionException(
955             Bytes.toStringBinary(encodedRegionName));
956         ure.initCause(ioe);
957         throw ure;
958       }
959     }
960   }
961 
962   public void createTable(HTableDescriptor hTableDescriptor,
963     byte [][] splitKeys)
964   throws IOException {
965     if (!isMasterRunning()) {
966       throw new MasterNotRunningException();
967     }
968 
969     HRegionInfo [] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
970     if (cpHost != null) {
971       cpHost.preCreateTable(hTableDescriptor, newRegions);
972     }
973 
974     this.executorService.submit(new CreateTableHandler(this,
975       this.fileSystemManager, this.serverManager, hTableDescriptor, conf,
976       newRegions, catalogTracker, assignmentManager));
977 
978     if (cpHost != null) {
979       cpHost.postCreateTable(hTableDescriptor, newRegions);
980     }
981   }
982 
983   private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
984     byte[][] splitKeys) {
985     HRegionInfo[] hRegionInfos = null;
986     if (splitKeys == null || splitKeys.length == 0) {
987       hRegionInfos = new HRegionInfo[]{
988           new HRegionInfo(hTableDescriptor.getName(), null, null)};
989     } else {
990       int numRegions = splitKeys.length + 1;
991       hRegionInfos = new HRegionInfo[numRegions];
992       byte[] startKey = null;
993       byte[] endKey = null;
994       for (int i = 0; i < numRegions; i++) {
995         endKey = (i == splitKeys.length) ? null : splitKeys[i];
996         hRegionInfos[i] =
997             new HRegionInfo(hTableDescriptor.getName(), startKey, endKey);
998         startKey = endKey;
999       }
1000     }
1001     return hRegionInfos;
1002   }
1003 
1004   private static boolean isCatalogTable(final byte [] tableName) {
1005     return Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) ||
1006            Bytes.equals(tableName, HConstants.META_TABLE_NAME);
1007   }
1008 
1009   @Override
1010   public void deleteTable(final byte [] tableName) throws IOException {
1011     if (cpHost != null) {
1012       cpHost.preDeleteTable(tableName);
1013     }
1014     this.executorService.submit(new DeleteTableHandler(tableName, this, this));
1015 
1016     if (cpHost != null) {
1017       cpHost.postDeleteTable(tableName);
1018     }
1019   }
1020 
1021   public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
1022   throws IOException {
1023     try {
1024       return this.assignmentManager.getReopenStatus(tableName);
1025     } catch (InterruptedException e) {
1026       throw new IOException("Interrupted", e);
1027     }
1028   }
1029 
1030   public void addColumn(byte [] tableName, HColumnDescriptor column)
1031   throws IOException {
1032     if (cpHost != null) {
1033       if (cpHost.preAddColumn(tableName, column)) {
1034         return;
1035       }
1036     }
1037     new TableAddFamilyHandler(tableName, column, this, this).process();
1038     if (cpHost != null) {
1039       cpHost.postAddColumn(tableName, column);
1040     }
1041   }
1042 
1043   public void modifyColumn(byte [] tableName, HColumnDescriptor descriptor)
1044   throws IOException {
1045     if (cpHost != null) {
1046       if (cpHost.preModifyColumn(tableName, descriptor)) {
1047         return;
1048       }
1049     }
1050     new TableModifyFamilyHandler(tableName, descriptor, this, this).process();
1051     if (cpHost != null) {
1052       cpHost.postModifyColumn(tableName, descriptor);
1053     }
1054   }
1055 
1056   public void deleteColumn(final byte [] tableName, final byte [] c)
1057   throws IOException {
1058     if (cpHost != null) {
1059       if (cpHost.preDeleteColumn(tableName, c)) {
1060         return;
1061       }
1062     }
1063     new TableDeleteFamilyHandler(tableName, c, this, this).process();
1064     if (cpHost != null) {
1065       cpHost.postDeleteColumn(tableName, c);
1066     }
1067   }
1068 
1069   public void enableTable(final byte [] tableName) throws IOException {
1070     if (cpHost != null) {
1071       cpHost.preEnableTable(tableName);
1072     }
1073     this.executorService.submit(new EnableTableHandler(this, tableName,
1074       catalogTracker, assignmentManager, false));
1075 
1076     if (cpHost != null) {
1077       cpHost.postEnableTable(tableName);
1078     }
1079   }
1080 
1081   public void disableTable(final byte [] tableName) throws IOException {
1082     if (cpHost != null) {
1083       cpHost.preDisableTable(tableName);
1084     }
1085     this.executorService.submit(new DisableTableHandler(this, tableName,
1086       catalogTracker, assignmentManager, false));
1087 
1088     if (cpHost != null) {
1089       cpHost.postDisableTable(tableName);
1090     }
1091   }
1092 
1093   /**
1094    * Return the region and current deployment for the region containing
1095    * the given row. If the region cannot be found, returns null. If it
1096    * is found, but not currently deployed, the second element of the pair
1097    * may be null.
1098    */
1099   Pair<HRegionInfo, ServerName> getTableRegionForRow(
1100       final byte [] tableName, final byte [] rowKey)
1101   throws IOException {
1102     final AtomicReference<Pair<HRegionInfo, ServerName>> result =
1103       new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
1104 
1105     MetaScannerVisitor visitor =
1106       new MetaScannerVisitor() {
1107         @Override
1108         public boolean processRow(Result data) throws IOException {
1109           if (data == null || data.size() <= 0) {
1110             return true;
1111           }
1112           Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(data);
1113           if (pair == null) {
1114             return false;
1115           }
1116           if (!Bytes.equals(pair.getFirst().getTableName(), tableName)) {
1117             return false;
1118           }
1119           result.set(pair);
1120           return true;
1121         }
1122     };
1123 
1124     MetaScanner.metaScan(conf, visitor, tableName, rowKey, 1);
1125     return result.get();
1126   }
1127 
1128   @Override
1129   public void modifyTable(final byte[] tableName, HTableDescriptor htd)
1130   throws IOException {
1131     if (cpHost != null) {
1132       cpHost.preModifyTable(tableName, htd);
1133     }
1134 
1135     this.executorService.submit(new ModifyTableHandler(tableName, htd, this,
1136       this));
1137 
1138     if (cpHost != null) {
1139       cpHost.postModifyTable(tableName, htd);
1140     }
1141   }
1142 
1143   @Override
1144   public void checkTableModifiable(final byte [] tableName)
1145   throws IOException {
1146     String tableNameStr = Bytes.toString(tableName);
1147     if (isCatalogTable(tableName)) {
1148       throw new IOException("Can't modify catalog tables");
1149     }
1150     if (!MetaReader.tableExists(getCatalogTracker(), tableNameStr)) {
1151       throw new TableNotFoundException(tableNameStr);
1152     }
1153     if (!getAssignmentManager().getZKTable().
1154         isDisabledTable(Bytes.toString(tableName))) {
1155       throw new TableNotDisabledException(tableName);
1156     }
1157   }
1158 
1159   public void clearFromTransition(HRegionInfo hri) {
1160     if (this.assignmentManager.isRegionInTransition(hri) != null) {
1161       this.assignmentManager.clearRegionFromTransition(hri);
1162     }
1163   }
1164   /**
1165    * @return cluster status
1166    */
1167   public ClusterStatus getClusterStatus() {
1168     return new ClusterStatus(VersionInfo.getVersion(),
1169       this.fileSystemManager.getClusterId(),
1170       this.serverManager.getOnlineServers(),
1171       this.serverManager.getDeadServers(),
1172       this.assignmentManager.getRegionsInTransition(),
1173       this.getCoprocessors());
1174   }
1175 
1176   public String getClusterId() {
1177     return fileSystemManager.getClusterId();
1178   }
1179 
1180   /**
1181    * The set of loaded coprocessors is stored in a static set. Since it's
1182    * statically allocated, it does not require that HMaster's cpHost be
1183    * initialized prior to accessing it.
1184    * @return a String representation of the set of names of the loaded
1185    * coprocessors.
1186    */
1187   public static String getLoadedCoprocessors() {
1188     return CoprocessorHost.getLoadedCoprocessors().toString();
1189   }
1190 
1191   /**
1192    * @return array of coprocessor SimpleNames.
1193    */
1194   public String[] getCoprocessors() {
1195     Set<String> masterCoprocessors =
1196         getCoprocessorHost().getCoprocessors();
1197     return masterCoprocessors.toArray(new String[0]);
1198   }
1199 
1200   @Override
1201   public void abort(final String msg, final Throwable t) {
1202     if (cpHost != null) {
1203       // HBASE-4014: dump a list of loaded coprocessors.
1204       LOG.fatal("Master server abort: loaded coprocessors are: " +
1205           getLoadedCoprocessors());
1206     }
1207 
1208     if (abortNow(msg, t)) {
1209       if (t != null) LOG.fatal(msg, t);
1210       else LOG.fatal(msg);
1211       this.abort = true;
1212       stop("Aborting");
1213     }
1214   }
1215 
1216   /**
1217    * We do the following.
1218    * 1. Create a new ZK session. (since our current one is expired)
1219    * 2. Try to become a primary master again
1220    * 3. Initialize all ZK based system trackers.
1221    * 4. Assign root and meta. (they are already assigned, but we need to update our
1222    * internal memory state to reflect it)
1223    * 5. Process any RIT if any during the process of our recovery.
1224    *
1225    * @return True if we could successfully recover from ZK session expiry.
1226    * @throws InterruptedException
1227    * @throws IOException
1228    */
1229   private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
1230       IOException, KeeperException {
1231     this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":"
1232         + this.serverName.getPort(), this, true);
1233 
1234     MonitoredTask status = 
1235       TaskMonitor.get().createStatus("Recovering expired ZK session");
1236     try {
1237       if (!becomeActiveMaster(status)) {
1238         return false;
1239       }
1240       initializeZKBasedSystemTrackers();
1241       // Update in-memory structures to reflect our earlier Root/Meta assignment.
1242       assignRootAndMeta(status);
1243       // process RIT if any
1244       // TODO: Why does this not call AssignmentManager.joinCluster?  Otherwise
1245       // we are not processing dead servers if any.
1246       this.assignmentManager.processDeadServersAndRegionsInTransition();
1247       return true;
1248     } finally {
1249       status.cleanup();
1250     }
1251   }
1252 
1253   /**
1254    * Check to see if the current trigger for abort is due to ZooKeeper session
1255    * expiry, and If yes, whether we can recover from ZK session expiry.
1256    *
1257    * @param msg Original abort message
1258    * @param t   The cause for current abort request
1259    * @return true if we should proceed with abort operation, false other wise.
1260    */
1261   private boolean abortNow(final String msg, final Throwable t) {
1262     if (!this.isActiveMaster) {
1263       return true;
1264     }
1265     if (t != null && t instanceof KeeperException.SessionExpiredException) {
1266       try {
1267         LOG.info("Primary Master trying to recover from ZooKeeper session " +
1268             "expiry.");
1269         return !tryRecoveringExpiredZKSession();
1270       } catch (Throwable newT) {
1271         LOG.error("Primary master encountered unexpected exception while " +
1272             "trying to recover from ZooKeeper session" +
1273             " expiry. Proceeding with server abort.", newT);
1274       }
1275     }
1276     return true;
1277   }
1278 
1279   @Override
1280   public ZooKeeperWatcher getZooKeeper() {
1281     return zooKeeper;
1282   }
1283 
1284   public MasterCoprocessorHost getCoprocessorHost() {
1285     return cpHost;
1286   }
1287 
1288   @Override
1289   public ServerName getServerName() {
1290     return this.serverName;
1291   }
1292 
1293   @Override
1294   public CatalogTracker getCatalogTracker() {
1295     return catalogTracker;
1296   }
1297 
1298   @Override
1299   public AssignmentManager getAssignmentManager() {
1300     return this.assignmentManager;
1301   }
1302   
1303   public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
1304     return rsFatals;
1305   }
1306 
1307   @Override
1308   public void shutdown() {
1309     if (cpHost != null) {
1310       try {
1311         cpHost.preShutdown();
1312       } catch (IOException ioe) {
1313         LOG.error("Error call master coprocessor preShutdown()", ioe);
1314       }
1315     }
1316     this.assignmentManager.shutdown();
1317     this.serverManager.shutdownCluster();
1318     try {
1319       this.clusterStatusTracker.setClusterDown();
1320     } catch (KeeperException e) {
1321       LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
1322     }
1323   }
1324 
1325   @Override
1326   public void stopMaster() {
1327     if (cpHost != null) {
1328       try {
1329         cpHost.preStopMaster();
1330       } catch (IOException ioe) {
1331         LOG.error("Error call master coprocessor preStopMaster()", ioe);
1332       }
1333     }
1334     stop("Stopped by " + Thread.currentThread().getName());
1335   }
1336 
1337   @Override
1338   public void stop(final String why) {
1339     LOG.info(why);
1340     this.stopped = true;
1341     // If we are a backup master, we need to interrupt wait
1342     if (this.activeMasterManager != null) {
1343       synchronized (this.activeMasterManager.clusterHasActiveMaster) {
1344         this.activeMasterManager.clusterHasActiveMaster.notifyAll();
1345       }
1346     }
1347   }
1348 
1349   @Override
1350   public boolean isStopped() {
1351     return this.stopped;
1352   }
1353 
1354   public boolean isAborted() {
1355     return this.abort;
1356   }
1357   
1358   
1359   /**
1360    * Report whether this master is currently the active master or not.
1361    * If not active master, we are parked on ZK waiting to become active.
1362    *
1363    * This method is used for testing.
1364    *
1365    * @return true if active master, false if not.
1366    */
1367   public boolean isActiveMaster() {
1368     return isActiveMaster;
1369   }
1370 
1371   /**
1372    * Report whether this master has completed with its initialization and is
1373    * ready.  If ready, the master is also the active master.  A standby master
1374    * is never ready.
1375    *
1376    * This method is used for testing.
1377    *
1378    * @return true if master is ready to go, false if not.
1379    */
1380   public boolean isInitialized() {
1381     return initialized;
1382   }
1383   
1384   @Override
1385   @Deprecated
1386   public void assign(final byte[] regionName, final boolean force)
1387       throws IOException {
1388     assign(regionName);
1389   }
1390 
1391   @Override
1392   public void assign(final byte [] regionName)throws IOException {
1393     Pair<HRegionInfo, ServerName> pair =
1394       MetaReader.getRegion(this.catalogTracker, regionName);
1395     if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
1396     if (cpHost != null) {
1397       if (cpHost.preAssign(pair.getFirst())) {
1398         return;
1399       }
1400     }
1401     assignRegion(pair.getFirst());
1402     if (cpHost != null) {
1403       cpHost.postAssign(pair.getFirst());
1404     }
1405   }
1406   
1407   
1408 
1409   public void assignRegion(HRegionInfo hri) {
1410     assignmentManager.assign(hri, true);
1411   }
1412 
1413   @Override
1414   public void unassign(final byte [] regionName, final boolean force)
1415   throws IOException {
1416     Pair<HRegionInfo, ServerName> pair =
1417       MetaReader.getRegion(this.catalogTracker, regionName);
1418     if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
1419     HRegionInfo hri = pair.getFirst();
1420     if (cpHost != null) {
1421       if (cpHost.preUnassign(hri, force)) {
1422         return;
1423       }
1424     }
1425     if (force) {
1426       this.assignmentManager.clearRegionFromTransition(hri);
1427       assignRegion(hri);
1428     } else {
1429       this.assignmentManager.unassign(hri, force);
1430     }
1431     if (cpHost != null) {
1432       cpHost.postUnassign(hri, force);
1433     }
1434   }
1435 
1436   /**
1437    * Get HTD array for given tables 
1438    * @param tableNames
1439    * @return HTableDescriptor[]
1440    */
1441   public HTableDescriptor[] getHTableDescriptors(List<String> tableNames) {
1442     List<HTableDescriptor> list =
1443       new ArrayList<HTableDescriptor>(tableNames.size());
1444     for (String s: tableNames) {
1445       HTableDescriptor htd = null;
1446       try {
1447         htd = this.tableDescriptors.get(s);
1448       } catch (IOException e) {
1449         LOG.warn("Failed getting descriptor for " + s, e);
1450       }
1451       if (htd == null) continue;
1452       list.add(htd);
1453     }
1454     return list.toArray(new HTableDescriptor [] {});
1455   }
1456 
1457   /**
1458    * Get all table descriptors
1459    * @return All descriptors or null if none.
1460    */
1461   public HTableDescriptor [] getHTableDescriptors() {
1462     Map<String, HTableDescriptor> descriptors = null;
1463     try {
1464       descriptors = this.tableDescriptors.getAll();
1465     } catch (IOException e) {
1466       LOG.warn("Failed getting all descriptors", e);
1467     }
1468     return descriptors == null?
1469       null: descriptors.values().toArray(new HTableDescriptor [] {});
1470   }
1471 
1472   /**
1473    * Compute the average load across all region servers.
1474    * Currently, this uses a very naive computation - just uses the number of
1475    * regions being served, ignoring stats about number of requests.
1476    * @return the average load
1477    */
1478   public double getAverageLoad() {
1479     return this.assignmentManager.getAverageLoad();
1480   }
1481 
1482   /**
1483    * Utility for constructing an instance of the passed HMaster class.
1484    * @param masterClass
1485    * @param conf
1486    * @return HMaster instance.
1487    */
1488   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
1489       final Configuration conf)  {
1490     try {
1491       Constructor<? extends HMaster> c =
1492         masterClass.getConstructor(Configuration.class);
1493       return c.newInstance(conf);
1494     } catch (InvocationTargetException ite) {
1495       Throwable target = ite.getTargetException() != null?
1496         ite.getTargetException(): ite;
1497       if (target.getCause() != null) target = target.getCause();
1498       throw new RuntimeException("Failed construction of Master: " +
1499         masterClass.toString(), target);
1500     } catch (Exception e) {
1501       throw new RuntimeException("Failed construction of Master: " +
1502         masterClass.toString() + ((e.getCause() != null)?
1503           e.getCause().getMessage(): ""), e);
1504     }
1505   }
1506 
1507   /**
1508    * @see org.apache.hadoop.hbase.master.HMasterCommandLine
1509    */
1510   public static void main(String [] args) throws Exception {
1511 	VersionInfo.logVersion();
1512     new HMasterCommandLine(HMaster.class).doMain(args);
1513   }
1514 }