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.lang.reflect.Method;
26  import java.net.InetAddress;
27  import java.net.InetSocketAddress;
28  import java.util.ArrayList;
29  import java.util.Collection;
30  import java.util.Collections;
31  import java.util.Comparator;
32  import java.util.HashMap;
33  import java.util.Iterator;
34  import java.util.List;
35  import java.util.Map;
36  import java.util.Set;
37  import java.util.concurrent.Callable;
38  import java.util.concurrent.ExecutionException;
39  import java.util.concurrent.Executors;
40  import java.util.concurrent.Future;
41  import java.util.concurrent.TimeUnit;
42  import java.util.concurrent.atomic.AtomicReference;
43  
44  import javax.management.ObjectName;
45  
46  import org.apache.commons.logging.Log;
47  import org.apache.commons.logging.LogFactory;
48  import org.apache.hadoop.conf.Configuration;
49  import org.apache.hadoop.fs.Path;
50  import org.apache.hadoop.hbase.Chore;
51  import org.apache.hadoop.hbase.ClusterStatus;
52  import org.apache.hadoop.hbase.HColumnDescriptor;
53  import org.apache.hadoop.hbase.HConstants;
54  import org.apache.hadoop.hbase.HRegionInfo;
55  import org.apache.hadoop.hbase.HServerLoad;
56  import org.apache.hadoop.hbase.HTableDescriptor;
57  import org.apache.hadoop.hbase.HealthCheckChore;
58  import org.apache.hadoop.hbase.MasterNotRunningException;
59  import org.apache.hadoop.hbase.PleaseHoldException;
60  import org.apache.hadoop.hbase.Server;
61  import org.apache.hadoop.hbase.ServerName;
62  import org.apache.hadoop.hbase.TableDescriptors;
63  import org.apache.hadoop.hbase.TableNotDisabledException;
64  import org.apache.hadoop.hbase.TableNotFoundException;
65  import org.apache.hadoop.hbase.UnknownRegionException;
66  import org.apache.hadoop.hbase.catalog.CatalogTracker;
67  import org.apache.hadoop.hbase.catalog.MetaReader;
68  import org.apache.hadoop.hbase.client.HConnectionManager;
69  import org.apache.hadoop.hbase.client.MetaScanner;
70  import org.apache.hadoop.hbase.client.UserProvider;
71  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
72  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
73  import org.apache.hadoop.hbase.client.Result;
74  import org.apache.hadoop.hbase.client.coprocessor.Exec;
75  import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
76  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
77  import org.apache.hadoop.hbase.executor.ExecutorService;
78  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
79  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
80  import org.apache.hadoop.hbase.ipc.HBaseRPC;
81  import org.apache.hadoop.hbase.ipc.HBaseServer;
82  import org.apache.hadoop.hbase.ipc.HMasterInterface;
83  import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
84  import org.apache.hadoop.hbase.ipc.ProtocolSignature;
85  import org.apache.hadoop.hbase.ipc.RpcServer;
86  import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
87  import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
88  import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
89  import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
90  import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
91  import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
92  import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
93  import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
94  import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
95  import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
96  import org.apache.hadoop.hbase.master.handler.TableEventHandler;
97  import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
98  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
99  import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
100 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
101 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
102 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
103 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
104 import org.apache.hadoop.hbase.regionserver.wal.HLog;
105 import org.apache.hadoop.hbase.replication.regionserver.Replication;
106 import org.apache.hadoop.hbase.security.User;
107 import org.apache.hadoop.hbase.snapshot.HSnapshotDescription;
108 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
109 import org.apache.hadoop.hbase.util.Bytes;
110 import org.apache.hadoop.hbase.util.FSTableDescriptors;
111 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
112 import org.apache.hadoop.hbase.util.HasThread;
113 import org.apache.hadoop.hbase.util.InfoServer;
114 import org.apache.hadoop.hbase.util.Pair;
115 import org.apache.hadoop.hbase.util.Sleeper;
116 import org.apache.hadoop.hbase.util.Strings;
117 import org.apache.hadoop.hbase.util.Threads;
118 import org.apache.hadoop.hbase.util.VersionInfo;
119 import org.apache.hadoop.hbase.zookeeper.ClusterId;
120 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
121 import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
122 import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
123 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
124 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
125 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
126 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
127 import org.apache.hadoop.io.MapWritable;
128 import org.apache.hadoop.io.Text;
129 import org.apache.hadoop.metrics.util.MBeanUtil;
130 import org.apache.hadoop.net.DNS;
131 import org.apache.zookeeper.KeeperException;
132 import org.apache.zookeeper.Watcher;
133 
134 import com.google.common.collect.ClassToInstanceMap;
135 import com.google.common.collect.Maps;
136 import com.google.common.collect.MutableClassToInstanceMap;
137 import com.google.protobuf.ServiceException;
138 
139 /**
140  * HMaster is the "master server" for HBase. An HBase cluster has one active
141  * master.  If many masters are started, all compete.  Whichever wins goes on to
142  * run the cluster.  All others park themselves in their constructor until
143  * master or cluster shutdown or until the active master loses its lease in
144  * zookeeper.  Thereafter, all running master jostle to take over master role.
145  *
146  * <p>The Master can be asked shutdown the cluster. See {@link #shutdown()}.  In
147  * this case it will tell all regionservers to go down and then wait on them
148  * all reporting in that they are down.  This master will then shut itself down.
149  *
150  * <p>You can also shutdown just this master.  Call {@link #stopMaster()}.
151  *
152  * @see HMasterInterface
153  * @see HMasterRegionInterface
154  * @see Watcher
155  */
156 public class HMaster extends HasThread
157 implements HMasterInterface, HMasterRegionInterface, MasterServices,
158 Server {
159   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
160 
161   // MASTER is name of the webapp and the attribute name used stuffing this
162   //instance into web context.
163   public static final String MASTER = "master";
164 
165   // The configuration for the Master
166   private final Configuration conf;
167   // server for the web ui
168   private InfoServer infoServer;
169 
170   // Our zk client.
171   private ZooKeeperWatcher zooKeeper;
172   // Manager and zk listener for master election
173   private ActiveMasterManager activeMasterManager;
174   // Region server tracker
175   private RegionServerTracker regionServerTracker;
176   // Draining region server tracker
177   private DrainingServerTracker drainingServerTracker;
178 
179   // RPC server for the HMaster
180   private final RpcServer rpcServer;
181 
182   /**
183    * This servers address.
184    */
185   private final InetSocketAddress isa;
186 
187   // Metrics for the HMaster
188   private final MasterMetrics metrics;
189   // file system manager for the master FS operations
190   private MasterFileSystem fileSystemManager;
191 
192   // server manager to deal with region server info
193   private ServerManager serverManager;
194 
195   // manager of assignment nodes in zookeeper
196   AssignmentManager assignmentManager;
197   // manager of catalog regions
198   private CatalogTracker catalogTracker;
199   // Cluster status zk tracker and local setter
200   private ClusterStatusTracker clusterStatusTracker;
201 
202   // buffer for "fatal error" notices from region servers
203   // in the cluster. This is only used for assisting
204   // operations/debugging.
205   private MemoryBoundedLogMessageBuffer rsFatals;
206 
207   // This flag is for stopping this Master instance.  Its set when we are
208   // stopping or aborting
209   private volatile boolean stopped = false;
210   // Set on abort -- usually failure of our zk session.
211   private volatile boolean abort = false;
212   // flag set after we become the active master (used for testing)
213   private volatile boolean isActiveMaster = false;
214 
215   // flag set after we complete initialization once active,
216   // it is not private since it's used in unit tests
217   volatile boolean initialized = false;
218 
219   // flag set after we complete assignRootAndMeta.
220   private volatile boolean serverShutdownHandlerEnabled = false;
221   // flag to indicate that we should be handling meta hlogs differently for splitting
222   private volatile boolean shouldSplitMetaSeparately;
223 
224   // Instance of the hbase executor service.
225   ExecutorService executorService;
226 
227   private LoadBalancer balancer;
228   private Thread balancerChore;
229   // If 'true', the balancer is 'on'.  If 'false', the balancer will not run.
230   private volatile boolean balanceSwitch = true;
231 
232   private CatalogJanitor catalogJanitorChore;
233   private LogCleaner logCleaner;
234   private HFileCleaner hfileCleaner;
235 
236   private MasterCoprocessorHost cpHost;
237   private final ServerName serverName;
238 
239   private TableDescriptors tableDescriptors;
240 
241   // Time stamps for when a hmaster was started and when it became active
242   private long masterStartTime;
243   private long masterActiveTime;
244 
245   // monitor for snapshot of hbase tables
246   private SnapshotManager snapshotManager;
247 
248   /**
249    * MX Bean for MasterInfo
250    */
251   private ObjectName mxBean = null;
252 
253   // Registered master protocol handlers
254   private ClassToInstanceMap<CoprocessorProtocol>
255       protocolHandlers = MutableClassToInstanceMap.create();
256 
257   private Map<String, Class<? extends CoprocessorProtocol>>
258       protocolHandlerNames = Maps.newHashMap();
259 
260   /** The health check chore. */
261   private HealthCheckChore healthCheckChore;
262 
263   /** flag when true, Master waits for log splitting complete before start up */
264   private boolean waitingOnLogSplitting = false;
265 
266   /** flag used in test cases in order to simulate RS failures during master initialization */
267   private volatile boolean initializationBeforeMetaAssignment = false;
268 
269   /** The following is used in master recovery scenario to re-register listeners */
270   private List<ZooKeeperListener> registeredZKListenersBeforeRecovery;
271 
272   /**
273    * Initializes the HMaster. The steps are as follows:
274    * <p>
275    * <ol>
276    * <li>Initialize HMaster RPC and address
277    * <li>Connect to ZooKeeper.
278    * </ol>
279    * <p>
280    * Remaining steps of initialization occur in {@link #run()} so that they
281    * run in their own thread rather than within the context of the constructor.
282    * @throws InterruptedException
283    */
284   public HMaster(final Configuration conf)
285   throws IOException, KeeperException, InterruptedException {
286     this.conf = new Configuration(conf);
287     // Disable the block cache on the master
288     this.conf.setFloat(HConstants.HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
289     // Set how many times to retry talking to another server over HConnection.
290     HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG);
291     // Server to handle client requests.
292     String hostname = Strings.domainNamePointerToHostName(DNS.getDefaultHost(
293       conf.get("hbase.master.dns.interface", "default"),
294       conf.get("hbase.master.dns.nameserver", "default")));
295     int port = conf.getInt(HConstants.MASTER_PORT, HConstants.DEFAULT_MASTER_PORT);
296     // Test that the hostname is reachable
297     InetSocketAddress initialIsa = new InetSocketAddress(hostname, port);
298     if (initialIsa.getAddress() == null) {
299       throw new IllegalArgumentException("Failed resolve of hostname " + initialIsa);
300     }
301     // Verify that the bind address is reachable if set
302     String bindAddress = conf.get("hbase.master.ipc.address");
303     if (bindAddress != null) {
304       initialIsa = new InetSocketAddress(bindAddress, port);
305       if (initialIsa.getAddress() == null) {
306         throw new IllegalArgumentException("Failed resolve of bind address " + initialIsa);
307       }
308     }
309     int numHandlers = conf.getInt("hbase.master.handler.count",
310       conf.getInt("hbase.regionserver.handler.count", 25));
311     this.rpcServer = HBaseRPC.getServer(this,
312       new Class<?>[]{HMasterInterface.class, HMasterRegionInterface.class},
313         initialIsa.getHostName(), // This is bindAddress if set else it's hostname
314         initialIsa.getPort(),
315         numHandlers,
316         0, // we dont use high priority handlers in master
317         conf.getBoolean("hbase.rpc.verbose", false), conf,
318         0); // this is a DNC w/o high priority handlers
319     // Set our address.
320     this.isa = this.rpcServer.getListenerAddress();
321     // We don't want to pass isa's hostname here since it could be 0.0.0.0
322     this.serverName = new ServerName(hostname,
323       this.isa.getPort(), System.currentTimeMillis());
324     this.rsFatals = new MemoryBoundedLogMessageBuffer(
325         conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
326 
327     // login the zookeeper client principal (if using security)
328     ZKUtil.loginClient(this.conf, "hbase.zookeeper.client.keytab.file",
329       "hbase.zookeeper.client.kerberos.principal", this.isa.getHostName());
330 
331     // initialize server principal (if using secure Hadoop)
332     UserProvider provider = UserProvider.instantiate(conf);
333     provider.login("hbase.master.keytab.file",
334       "hbase.master.kerberos.principal", this.isa.getHostName());
335 
336     // set the thread name now we have an address
337     setName(MASTER + "-" + this.serverName.toString());
338 
339     Replication.decorateMasterConfiguration(this.conf);
340 
341     // Hack! Maps DFSClient => Master for logs.  HDFS made this
342     // config param for task trackers, but we can piggyback off of it.
343     if (this.conf.get("mapred.task.id") == null) {
344       this.conf.set("mapred.task.id", "hb_m_" + this.serverName.toString());
345     }
346 
347     this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":" + isa.getPort(), this, true);
348     this.rpcServer.startThreads();
349     this.metrics = new MasterMetrics(getServerName().toString());
350 
351     // Health checker thread.
352     int sleepTime = this.conf.getInt(HConstants.HEALTH_CHORE_WAKE_FREQ,
353       HConstants.DEFAULT_THREAD_WAKE_FREQUENCY);
354     if (isHealthCheckerConfigured()) {
355       healthCheckChore = new HealthCheckChore(sleepTime, this, getConfiguration());
356     }
357 
358     this.shouldSplitMetaSeparately = conf.getBoolean(HLog.SEPARATE_HLOG_FOR_META, false);
359     waitingOnLogSplitting = this.conf.getBoolean("hbase.master.wait.for.log.splitting", false);
360   }
361 
362   /**
363    * Stall startup if we are designated a backup master; i.e. we want someone
364    * else to become the master before proceeding.
365    * @param c
366    * @param amm
367    * @throws InterruptedException
368    */
369   private static void stallIfBackupMaster(final Configuration c,
370       final ActiveMasterManager amm)
371   throws InterruptedException {
372     // If we're a backup master, stall until a primary to writes his address
373     if (!c.getBoolean(HConstants.MASTER_TYPE_BACKUP,
374       HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
375       return;
376     }
377     LOG.debug("HMaster started in backup mode.  " +
378       "Stalling until master znode is written.");
379     // This will only be a minute or so while the cluster starts up,
380     // so don't worry about setting watches on the parent znode
381     while (!amm.isActiveMaster()) {
382       LOG.debug("Waiting for master address ZNode to be written " +
383         "(Also watching cluster state node)");
384       Thread.sleep(c.getInt("zookeeper.session.timeout", 180 * 1000));
385     }
386 
387   }
388 
389   /**
390    * Main processing loop for the HMaster.
391    * <ol>
392    * <li>Block until becoming active master
393    * <li>Finish initialization via finishInitialization(MonitoredTask)
394    * <li>Enter loop until we are stopped
395    * <li>Stop services and perform cleanup once stopped
396    * </ol>
397    */
398   @Override
399   public void run() {
400     MonitoredTask startupStatus =
401       TaskMonitor.get().createStatus("Master startup");
402     startupStatus.setDescription("Master startup");
403     masterStartTime = System.currentTimeMillis();
404     try {
405       this.registeredZKListenersBeforeRecovery = this.zooKeeper.getListeners();
406 
407       // Put up info server.
408       int port = this.conf.getInt("hbase.master.info.port", 60010);
409       if (port >= 0) {
410         String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
411         this.infoServer = new InfoServer(MASTER, a, port, false, this.conf);
412         this.infoServer.addServlet("status", "/master-status", MasterStatusServlet.class);
413         this.infoServer.addServlet("dump", "/dump", MasterDumpServlet.class);
414         this.infoServer.setAttribute(MASTER, this);
415         this.infoServer.start();
416       }
417 
418       /*
419        * Block on becoming the active master.
420        *
421        * We race with other masters to write our address into ZooKeeper.  If we
422        * succeed, we are the primary/active master and finish initialization.
423        *
424        * If we do not succeed, there is another active master and we should
425        * now wait until it dies to try and become the next active master.  If we
426        * do not succeed on our first attempt, this is no longer a cluster startup.
427        */
428       becomeActiveMaster(startupStatus);
429 
430       // We are either the active master or we were asked to shutdown
431       if (!this.stopped) {
432         finishInitialization(startupStatus, false);
433         loop();
434       }
435     } catch (Throwable t) {
436       // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
437       if (t instanceof NoClassDefFoundError &&
438           t.getMessage().contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) {
439           // improved error message for this special case
440           abort("HBase is having a problem with its Hadoop jars.  You may need to "
441               + "recompile HBase against Hadoop version "
442               +  org.apache.hadoop.util.VersionInfo.getVersion()
443               + " or change your hadoop jars to start properly", t);
444       } else {
445         abort("Unhandled exception. Starting shutdown.", t);
446       }
447     } finally {
448       startupStatus.cleanup();
449 
450       stopChores();
451       // Wait for all the remaining region servers to report in IFF we were
452       // running a cluster shutdown AND we were NOT aborting.
453       if (!this.abort && this.serverManager != null &&
454           this.serverManager.isClusterShutdown()) {
455         this.serverManager.letRegionServersShutdown();
456       }
457       stopServiceThreads();
458       // Stop services started for both backup and active masters
459       if (this.activeMasterManager != null) this.activeMasterManager.stop();
460       if (this.catalogTracker != null) this.catalogTracker.stop();
461       if (this.serverManager != null) this.serverManager.stop();
462       if (this.assignmentManager != null) this.assignmentManager.stop();
463       if (this.fileSystemManager != null) this.fileSystemManager.stop();
464       if (this.snapshotManager != null) this.snapshotManager.stop("server shutting down.");
465       this.zooKeeper.close();
466     }
467     LOG.info("HMaster main thread exiting");
468   }
469 
470   /**
471    * Try becoming active master.
472    * @param startupStatus
473    * @return True if we could successfully become the active master.
474    * @throws InterruptedException
475    */
476   private boolean becomeActiveMaster(MonitoredTask startupStatus)
477   throws InterruptedException {
478     // TODO: This is wrong!!!! Should have new servername if we restart ourselves,
479     // if we come back to life.
480     this.activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName,
481         this);
482     this.zooKeeper.registerListener(activeMasterManager);
483     stallIfBackupMaster(this.conf, this.activeMasterManager);
484 
485     // The ClusterStatusTracker is setup before the other
486     // ZKBasedSystemTrackers because it's needed by the activeMasterManager
487     // to check if the cluster should be shutdown.
488     this.clusterStatusTracker = new ClusterStatusTracker(getZooKeeper(), this);
489     this.clusterStatusTracker.start();
490     return this.activeMasterManager.blockUntilBecomingActiveMaster(startupStatus,
491         this.clusterStatusTracker);
492   }
493 
494   /**
495    * Initialize all ZK based system trackers.
496    * @throws IOException
497    * @throws InterruptedException
498    */
499   private void initializeZKBasedSystemTrackers() throws IOException,
500       InterruptedException, KeeperException {
501     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.conf, this);
502     this.catalogTracker.start();
503 
504     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
505     this.assignmentManager = new AssignmentManager(this, serverManager,
506         this.catalogTracker, this.balancer, this.executorService);
507     zooKeeper.registerListenerFirst(assignmentManager);
508 
509     this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
510         this.serverManager);
511     this.regionServerTracker.start();
512 
513     this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this,
514       this.serverManager);
515     this.drainingServerTracker.start();
516 
517     // Set the cluster as up.  If new RSs, they'll be waiting on this before
518     // going ahead with their startup.
519     boolean wasUp = this.clusterStatusTracker.isClusterUp();
520     if (!wasUp) this.clusterStatusTracker.setClusterUp();
521 
522     LOG.info("Server active/primary master; " + this.serverName +
523         ", sessionid=0x" +
524         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
525         ", cluster-up flag was=" + wasUp);
526 
527     // create the snapshot manager
528     this.snapshotManager = new SnapshotManager(this, this.metrics);
529   }
530 
531   // Check if we should stop every second.
532   private Sleeper stopSleeper = new Sleeper(1000, this);
533   private void loop() {
534     while (!this.stopped) {
535       stopSleeper.sleep();
536     }
537   }
538 
539   /**
540    * Finish initialization of HMaster after becoming the primary master.
541    *
542    * <ol>
543    * <li>Initialize master components - file system manager, server manager,
544    *     assignment manager, region server tracker, catalog tracker, etc</li>
545    * <li>Start necessary service threads - rpc server, info server,
546    *     executor services, etc</li>
547    * <li>Set cluster as UP in ZooKeeper</li>
548    * <li>Wait for RegionServers to check-in</li>
549    * <li>Split logs and perform data recovery, if necessary</li>
550    * <li>Ensure assignment of root and meta regions<li>
551    * <li>Handle either fresh cluster start or master failover</li>
552    * </ol>
553    * @param masterRecovery
554    *
555    * @throws IOException
556    * @throws InterruptedException
557    * @throws KeeperException
558    */
559   private void finishInitialization(MonitoredTask status, boolean masterRecovery)
560   throws IOException, InterruptedException, KeeperException {
561 
562     isActiveMaster = true;
563 
564     /*
565      * We are active master now... go initialize components we need to run.
566      * Note, there may be dross in zk from previous runs; it'll get addressed
567      * below after we determine if cluster startup or failover.
568      */
569 
570     status.setStatus("Initializing Master file system");
571     this.masterActiveTime = System.currentTimeMillis();
572     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
573     this.fileSystemManager = new MasterFileSystem(this, this, metrics, masterRecovery);
574 
575     this.tableDescriptors =
576       new FSTableDescriptors(this.fileSystemManager.getFileSystem(),
577       this.fileSystemManager.getRootDir());
578 
579     // publish cluster ID
580     status.setStatus("Publishing Cluster ID in ZooKeeper");
581     ClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
582     if (!masterRecovery) {
583       this.executorService = new ExecutorService(getServerName().toString());
584       this.serverManager = new ServerManager(this, this);
585     }
586 
587 
588     status.setStatus("Initializing ZK system trackers");
589     initializeZKBasedSystemTrackers();
590 
591     if (!masterRecovery) {
592       // initialize master side coprocessors before we start handling requests
593       status.setStatus("Initializing master coprocessors");
594       this.cpHost = new MasterCoprocessorHost(this, this.conf);
595 
596       // start up all service threads.
597       status.setStatus("Initializing master service threads");
598       startServiceThreads();
599     }
600 
601     // Wait for region servers to report in.
602     this.serverManager.waitForRegionServers(status);
603     // Check zk for regionservers that are up but didn't register
604     for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
605       if (!this.serverManager.isServerOnline(sn)) {
606         // Not registered; add it.
607         LOG.info("Registering server found up in zk but who has not yet " +
608           "reported in: " + sn);
609         this.serverManager.recordNewServer(sn, HServerLoad.EMPTY_HSERVERLOAD);
610       }
611     }
612     if (!masterRecovery) {
613       this.assignmentManager.startTimeOutMonitor();
614     }
615 
616     // get a list for previously failed RS which need recovery work
617     Set<ServerName> failedServers = this.fileSystemManager.getFailedServersFromLogFolders();
618     if (waitingOnLogSplitting) {
619       List<ServerName> servers = new ArrayList<ServerName>(failedServers);
620       this.fileSystemManager.splitAllLogs(servers);
621       failedServers.clear();
622     }
623 
624     ServerName preRootServer = this.catalogTracker.getRootLocation();
625     if (preRootServer != null && failedServers.contains(preRootServer)) {
626       // create recovered edits file for _ROOT_ server
627       this.fileSystemManager.splitAllLogs(preRootServer);
628       failedServers.remove(preRootServer);
629     }
630 
631     this.initializationBeforeMetaAssignment = true;
632     // Make sure root assigned before proceeding.
633     if (!assignRoot(status)) return;
634 
635     // SSH should enabled for ROOT before META region assignment
636     // because META region assignment is depending on ROOT server online.
637     this.serverManager.enableSSHForRoot();
638 
639     // log splitting for .META. server
640     ServerName preMetaServer = this.catalogTracker.getMetaLocationOrReadLocationFromRoot();
641     if (preMetaServer != null && failedServers.contains(preMetaServer)) {
642       // create recovered edits file for .META. server
643       this.fileSystemManager.splitAllLogs(preMetaServer);
644       failedServers.remove(preMetaServer);
645     }
646 
647     // Make sure meta assigned before proceeding.
648     if (!assignMeta(status, ((masterRecovery) ? null : preMetaServer), preRootServer)) return;
649 
650     enableServerShutdownHandler();
651 
652     // handle other dead servers in SSH
653     status.setStatus("Submit log splitting work of non-meta region servers");
654     for (ServerName curServer : failedServers) {
655       this.serverManager.expireServer(curServer);
656     }
657 
658     // Update meta with new HRI if required. i.e migrate all HRI with HTD to
659     // HRI with out HTD in meta and update the status in ROOT. This must happen
660     // before we assign all user regions or else the assignment will fail.
661     // TODO: Remove this when we do 0.94.
662     org.apache.hadoop.hbase.catalog.MetaMigrationRemovingHTD.
663       updateMetaWithNewHRI(this);
664 
665     // Fixup assignment manager status
666     status.setStatus("Starting assignment manager");
667     this.assignmentManager.joinCluster();
668 
669     this.balancer.setClusterStatus(getClusterStatus());
670     this.balancer.setMasterServices(this);
671 
672     // Fixing up missing daughters if any
673     status.setStatus("Fixing up missing daughters");
674     fixupDaughters(status);
675 
676     if (!masterRecovery) {
677       // Start balancer and meta catalog janitor after meta and regions have
678       // been assigned.
679       status.setStatus("Starting balancer and catalog janitor");
680       this.balancerChore = getAndStartBalancerChore(this);
681       this.catalogJanitorChore = new CatalogJanitor(this, this);
682       startCatalogJanitorChore();
683       registerMBean();
684     }
685 
686     status.markComplete("Initialization successful");
687     LOG.info("Master has completed initialization");
688     initialized = true;
689 
690     // clear the dead servers with same host name and port of online server because we are not
691     // removing dead server with same hostname and port of rs which is trying to check in before
692     // master initialization. See HBASE-5916.
693     this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
694 
695     if (!masterRecovery) {
696       if (this.cpHost != null) {
697         // don't let cp initialization errors kill the master
698         try {
699           this.cpHost.postStartMaster();
700         } catch (IOException ioe) {
701           LOG.error("Coprocessor postStartMaster() hook failed", ioe);
702         }
703       }
704     }
705   }
706 
707   /**
708    * If ServerShutdownHandler is disabled, we enable it and expire those dead
709    * but not expired servers.
710    *
711    * @throws IOException
712    */
713   private void enableServerShutdownHandler() throws IOException {
714     if (!serverShutdownHandlerEnabled) {
715       serverShutdownHandlerEnabled = true;
716       this.serverManager.expireDeadNotExpiredServers();
717     }
718   }
719 
720   /**
721    * Useful for testing purpose also where we have
722    * master restart scenarios.
723    */
724   protected void startCatalogJanitorChore() {
725     Threads.setDaemonThreadRunning(catalogJanitorChore.getThread());
726   }
727 
728   /**
729    * Check <code>-ROOT-</code> is assigned. If not, assign it.
730    * @param status MonitoredTask
731    * @throws InterruptedException
732    * @throws IOException
733    * @throws KeeperException
734    */
735   private boolean assignRoot(MonitoredTask status)
736   throws InterruptedException, IOException, KeeperException {
737     int assigned = 0;
738     long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
739 
740     // Work on ROOT region.  Is it in zk in transition?
741     status.setStatus("Assigning ROOT region");
742     boolean rit = this.assignmentManager.
743       processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.ROOT_REGIONINFO);
744     ServerName currentRootServer = null;
745     boolean rootRegionLocation = catalogTracker.verifyRootRegionLocation(timeout);
746     if (!rit && !rootRegionLocation) {
747       currentRootServer = this.catalogTracker.getRootLocation();
748       splitLogAndExpireIfOnline(currentRootServer);
749       this.assignmentManager.assignRoot();
750       waitForRootAssignment();
751       if (!this.assignmentManager.isRegionAssigned(HRegionInfo.ROOT_REGIONINFO) || this.stopped) {
752         return false;
753       }
754       assigned++;
755     } else if (rit && !rootRegionLocation) {
756       waitForRootAssignment();
757       if (!this.assignmentManager.isRegionAssigned(HRegionInfo.ROOT_REGIONINFO) || this.stopped) {
758         return false;
759       }
760       assigned++;
761     } else {
762       // Region already assigned. We didn't assign it. Add to in-memory state.
763       this.assignmentManager.regionOnline(HRegionInfo.ROOT_REGIONINFO,
764           this.catalogTracker.getRootLocation());
765     }
766     // Enable the ROOT table if on process fail over the RS containing ROOT
767     // was active.
768     enableCatalogTables(Bytes.toString(HConstants.ROOT_TABLE_NAME));
769     LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit +
770       ", location=" + catalogTracker.getRootLocation());
771 
772     status.setStatus("ROOT assigned.");
773     return true;
774   }
775 
776   /**
777    * Check <code>.META.</code> is assigned. If not, assign it.
778    * @param status MonitoredTask
779    * @param previousMetaServer ServerName of previous meta region server before current start up
780    * @param previousRootServer ServerName of previous root region server before current start up
781    * @throws InterruptedException
782    * @throws IOException
783    * @throws KeeperException
784    */
785   private boolean assignMeta(MonitoredTask status, ServerName previousMetaServer,
786       ServerName previousRootServer)
787       throws InterruptedException,
788       IOException, KeeperException {
789     int assigned = 0;
790     long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
791 
792     status.setStatus("Assigning META region");
793     boolean rit =
794         this.assignmentManager
795             .processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
796     boolean metaRegionLocation = this.catalogTracker.verifyMetaRegionLocation(timeout);
797     if (!rit && !metaRegionLocation) {
798       ServerName currentMetaServer =
799           (previousMetaServer != null) ? previousMetaServer : this.catalogTracker
800               .getMetaLocationOrReadLocationFromRoot();
801       if (currentMetaServer != null && !currentMetaServer.equals(previousRootServer)) {
802         fileSystemManager.splitAllLogs(currentMetaServer);
803         if (this.serverManager.isServerOnline(currentMetaServer)) {
804           this.serverManager.expireServer(currentMetaServer);
805         }
806       }
807       assignmentManager.assignMeta();
808       enableSSHandWaitForMeta();
809       if (!this.assignmentManager.isRegionAssigned(HRegionInfo.FIRST_META_REGIONINFO)
810           || this.stopped) {
811         return false;
812       }
813       assigned++;
814     } else if (rit && !metaRegionLocation) {
815       enableSSHandWaitForMeta();
816       if (!this.assignmentManager.isRegionAssigned(HRegionInfo.FIRST_META_REGIONINFO)
817           || this.stopped) {
818         return false;
819       }
820       assigned++;
821     } else {
822       // Region already assigned. We didnt' assign it. Add to in-memory state.
823       this.assignmentManager.regionOnline(HRegionInfo.FIRST_META_REGIONINFO,
824         this.catalogTracker.getMetaLocation());
825     }
826     enableCatalogTables(Bytes.toString(HConstants.META_TABLE_NAME));
827     LOG.info(".META. assigned=" + assigned + ", rit=" + rit + ", location="
828         + catalogTracker.getMetaLocation());
829     status.setStatus("META assigned.");
830     return true;
831   }
832 
833   private void enableSSHandWaitForMeta() throws IOException,
834       InterruptedException {
835     enableServerShutdownHandler();
836     this.catalogTracker.waitForMeta();
837     // Above check waits for general meta availability but this does not
838     // guarantee that the transition has completed
839     this.assignmentManager
840         .waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
841   }
842 
843   private void waitForRootAssignment() throws InterruptedException, IOException {
844     // Enable SSH for ROOT to prevent a newly assigned ROOT crashes again before global SSH is
845     // enabled
846     this.serverManager.enableSSHForRoot();
847     this.catalogTracker.waitForRoot();
848     // This guarantees that the transition has completed
849     this.assignmentManager.waitForAssignment(HRegionInfo.ROOT_REGIONINFO);
850   }
851 
852   private void enableCatalogTables(String catalogTableName) {
853     if (!this.assignmentManager.getZKTable().isEnabledTable(catalogTableName)) {
854       this.assignmentManager.setEnabledTable(catalogTableName);
855     }
856   }
857 
858   void fixupDaughters(final MonitoredTask status) throws IOException, KeeperException {
859     final Map<HRegionInfo, Result> offlineSplitParents =
860       new HashMap<HRegionInfo, Result>();
861     // This visitor collects offline split parents in the .META. table
862     MetaReader.Visitor visitor = new MetaReader.Visitor() {
863       @Override
864       public boolean visit(Result r) throws IOException {
865         if (r == null || r.isEmpty()) return true;
866         HRegionInfo info =
867           MetaReader.parseHRegionInfoFromCatalogResult(
868             r, HConstants.REGIONINFO_QUALIFIER);
869         if (info == null) return true; // Keep scanning
870         if (info.isOffline() && info.isSplit()) {
871           offlineSplitParents.put(info, r);
872         }
873         // Returning true means "keep scanning"
874         return true;
875       }
876     };
877     // Run full scan of .META. catalog table passing in our custom visitor
878     MetaReader.fullScan(this.catalogTracker, visitor);
879     // Now work on our list of found parents. See if any we can clean up.
880     int fixups = 0;
881     for (Map.Entry<HRegionInfo, Result> e : offlineSplitParents.entrySet()) {
882       String node = ZKAssign.getNodeName(zooKeeper, e.getKey().getEncodedName());
883       byte[] data = ZKUtil.getData(zooKeeper, node);
884       if (data == null) { // otherwise, splitting is still going on, skip it
885         fixups += ServerShutdownHandler.fixupDaughters(
886           e.getValue(), assignmentManager, catalogTracker);
887       }
888     }
889     if (fixups != 0) {
890       LOG.info("Scanned the catalog and fixed up " + fixups +
891         " missing daughter region(s)");
892     }
893   }
894 
895   /**
896    * Expire a server if we find it is one of the online servers.
897    * @param sn ServerName to check.
898    * @throws IOException
899    */
900   private void splitLogAndExpireIfOnline(final ServerName sn)
901       throws IOException {
902     if (sn == null || !serverManager.isServerOnline(sn)) {
903       return;
904     }
905     LOG.info("Forcing splitLog and expire of " + sn);
906     if (this.shouldSplitMetaSeparately) {
907       fileSystemManager.splitMetaLog(sn);
908       fileSystemManager.splitLog(sn);
909     } else {
910       fileSystemManager.splitAllLogs(sn);
911     }
912     serverManager.expireServer(sn);
913   }
914 
915   @Override
916   public ProtocolSignature getProtocolSignature(
917       String protocol, long version, int clientMethodsHashCode)
918   throws IOException {
919     if (HMasterInterface.class.getName().equals(protocol)) {
920       return new ProtocolSignature(HMasterInterface.VERSION, null);
921     } else if (HMasterRegionInterface.class.getName().equals(protocol)) {
922       return new ProtocolSignature(HMasterRegionInterface.VERSION, null);
923     }
924     throw new IOException("Unknown protocol: " + protocol);
925   }
926 
927   public long getProtocolVersion(String protocol, long clientVersion) {
928     if (HMasterInterface.class.getName().equals(protocol)) {
929       return HMasterInterface.VERSION;
930     } else if (HMasterRegionInterface.class.getName().equals(protocol)) {
931       return HMasterRegionInterface.VERSION;
932     }
933     // unknown protocol
934     LOG.warn("Version requested for unimplemented protocol: "+protocol);
935     return -1;
936   }
937 
938   @Override
939   public TableDescriptors getTableDescriptors() {
940     return this.tableDescriptors;
941   }
942 
943   /** @return InfoServer object. Maybe null.*/
944   public InfoServer getInfoServer() {
945     return this.infoServer;
946   }
947 
948   @Override
949   public Configuration getConfiguration() {
950     return this.conf;
951   }
952 
953   @Override
954   public ServerManager getServerManager() {
955     return this.serverManager;
956   }
957 
958   @Override
959   public ExecutorService getExecutorService() {
960     return this.executorService;
961   }
962 
963   @Override
964   public MasterFileSystem getMasterFileSystem() {
965     return this.fileSystemManager;
966   }
967 
968   /**
969    * Get the ZK wrapper object - needed by master_jsp.java
970    * @return the zookeeper wrapper
971    */
972   public ZooKeeperWatcher getZooKeeperWatcher() {
973     return this.zooKeeper;
974   }
975 
976   public ActiveMasterManager getActiveMasterManager() {
977     return this.activeMasterManager;
978   }
979 
980   /*
981    * Start up all services. If any of these threads gets an unhandled exception
982    * then they just die with a logged message.  This should be fine because
983    * in general, we do not expect the master to get such unhandled exceptions
984    *  as OOMEs; it should be lightly loaded. See what HRegionServer does if
985    *  need to install an unexpected exception handler.
986    */
987   private void startServiceThreads() throws IOException{
988 
989    // Start the executor service pools
990    this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
991       conf.getInt("hbase.master.executor.openregion.threads", 5));
992    this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
993       conf.getInt("hbase.master.executor.closeregion.threads", 5));
994    this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
995       conf.getInt("hbase.master.executor.serverops.threads", 3));
996    this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
997       conf.getInt("hbase.master.executor.serverops.threads", 5));
998 
999    // We depend on there being only one instance of this executor running
1000    // at a time.  To do concurrency, would need fencing of enable/disable of
1001    // tables.
1002    this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
1003 
1004    // Start log cleaner thread
1005    String n = Thread.currentThread().getName();
1006    int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
1007    this.logCleaner =
1008       new LogCleaner(cleanerInterval,
1009          this, conf, getMasterFileSystem().getFileSystem(),
1010          getMasterFileSystem().getOldLogDir());
1011          Threads.setDaemonThreadRunning(logCleaner.getThread(), n + ".oldLogCleaner");
1012 
1013    //start the hfile archive cleaner thread
1014     Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
1015     this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem()
1016         .getFileSystem(), archiveDir);
1017     Threads.setDaemonThreadRunning(hfileCleaner.getThread(), n + ".archivedHFileCleaner");
1018 
1019    // Start the health checker
1020    if (this.healthCheckChore != null) {
1021      Threads.setDaemonThreadRunning(this.healthCheckChore.getThread(), n + ".healthChecker");
1022    }
1023 
1024     // Start allowing requests to happen.
1025     this.rpcServer.openServer();
1026     if (LOG.isDebugEnabled()) {
1027       LOG.debug("Started service threads");
1028     }
1029 
1030   }
1031 
1032   private void stopServiceThreads() {
1033     if (LOG.isDebugEnabled()) {
1034       LOG.debug("Stopping service threads");
1035     }
1036     if (this.rpcServer != null) this.rpcServer.stop();
1037     // Clean up and close up shop
1038     if (this.logCleaner!= null) this.logCleaner.interrupt();
1039     if (this.hfileCleaner != null) this.hfileCleaner.interrupt();
1040 
1041     if (this.infoServer != null) {
1042       LOG.info("Stopping infoServer");
1043       try {
1044         this.infoServer.stop();
1045       } catch (Exception ex) {
1046         ex.printStackTrace();
1047       }
1048     }
1049     if (this.executorService != null) this.executorService.shutdown();
1050     if (this.healthCheckChore != null) {
1051       this.healthCheckChore.interrupt();
1052     }
1053   }
1054 
1055   private static Thread getAndStartBalancerChore(final HMaster master) {
1056     String name = master.getServerName() + "-BalancerChore";
1057     int balancerPeriod =
1058       master.getConfiguration().getInt("hbase.balancer.period", 300000);
1059     // Start up the load balancer chore
1060     Chore chore = new Chore(name, balancerPeriod, master) {
1061       @Override
1062       protected void chore() {
1063         master.balance();
1064       }
1065     };
1066     return Threads.setDaemonThreadRunning(chore.getThread());
1067   }
1068 
1069   private void stopChores() {
1070     if (this.balancerChore != null) {
1071       this.balancerChore.interrupt();
1072     }
1073     if (this.catalogJanitorChore != null) {
1074       this.catalogJanitorChore.interrupt();
1075     }
1076   }
1077 
1078   @Override
1079   public MapWritable regionServerStartup(final int port,
1080     final long serverStartCode, final long serverCurrentTime)
1081   throws IOException {
1082     // Register with server manager
1083     InetAddress ia = HBaseServer.getRemoteIp();
1084     ServerName rs = this.serverManager.regionServerStartup(ia, port,
1085       serverStartCode, serverCurrentTime);
1086     // Send back some config info
1087     MapWritable mw = createConfigurationSubset();
1088     mw.put(new Text(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER),
1089       new Text(rs.getHostname()));
1090     return mw;
1091   }
1092 
1093   /**
1094    * @return Subset of configuration to pass initializing regionservers: e.g.
1095    * the filesystem to use and root directory to use.
1096    */
1097   protected MapWritable createConfigurationSubset() {
1098     MapWritable mw = addConfig(new MapWritable(), HConstants.HBASE_DIR);
1099     return addConfig(mw, "fs.default.name");
1100   }
1101 
1102   private MapWritable addConfig(final MapWritable mw, final String key) {
1103     mw.put(new Text(key), new Text(this.conf.get(key)));
1104     return mw;
1105   }
1106 
1107   @Override
1108   public void regionServerReport(final byte [] sn, final HServerLoad hsl)
1109   throws IOException {
1110     this.serverManager.regionServerReport(ServerName.parseVersionedServerName(sn), hsl);
1111     if (hsl != null && this.metrics != null) {
1112       // Up our metrics.
1113       this.metrics.incrementRequests(hsl.getTotalNumberOfRequests());
1114     }
1115   }
1116 
1117   @Override
1118   public void reportRSFatalError(byte [] sn, String errorText) {
1119     String msg = "Region server " + Bytes.toString(sn) +
1120       " reported a fatal error:\n" + errorText;
1121     LOG.error(msg);
1122     rsFatals.add(msg);
1123   }
1124 
1125   public boolean isMasterRunning() {
1126     return !isStopped();
1127   }
1128 
1129   /**
1130    * @return Maximum time we should run balancer for
1131    */
1132   private int getBalancerCutoffTime() {
1133     int balancerCutoffTime =
1134       getConfiguration().getInt("hbase.balancer.max.balancing", -1);
1135     if (balancerCutoffTime == -1) {
1136       // No time period set so create one -- do half of balancer period.
1137       int balancerPeriod =
1138         getConfiguration().getInt("hbase.balancer.period", 300000);
1139       balancerCutoffTime = balancerPeriod / 2;
1140       // If nonsense period, set it to balancerPeriod
1141       if (balancerCutoffTime <= 0) balancerCutoffTime = balancerPeriod;
1142     }
1143     return balancerCutoffTime;
1144   }
1145 
1146   @Override
1147   public boolean balance() {
1148     // if master not initialized, don't run balancer.
1149     if (!this.initialized) {
1150       LOG.debug("Master has not been initialized, don't run balancer.");
1151       return false;
1152     }
1153     // If balance not true, don't run balancer.
1154     if (!this.balanceSwitch) return false;
1155     // Do this call outside of synchronized block.
1156     int maximumBalanceTime = getBalancerCutoffTime();
1157     long cutoffTime = System.currentTimeMillis() + maximumBalanceTime;
1158     boolean balancerRan;
1159     synchronized (this.balancer) {
1160       // Only allow one balance run at at time.
1161       if (this.assignmentManager.isRegionsInTransition()) {
1162         LOG.debug("Not running balancer because " +
1163           this.assignmentManager.getRegionsInTransition().size() +
1164           " region(s) in transition: " +
1165           org.apache.commons.lang.StringUtils.
1166             abbreviate(this.assignmentManager.getRegionsInTransition().toString(), 256));
1167         return false;
1168       }
1169       if (this.serverManager.areDeadServersInProgress()) {
1170         LOG.debug("Not running balancer because processing dead regionserver(s): " +
1171           this.serverManager.getDeadServers());
1172         return false;
1173       }
1174 
1175       if (this.cpHost != null) {
1176         try {
1177           if (this.cpHost.preBalance()) {
1178             LOG.debug("Coprocessor bypassing balancer request");
1179             return false;
1180           }
1181         } catch (IOException ioe) {
1182           LOG.error("Error invoking master coprocessor preBalance()", ioe);
1183           return false;
1184         }
1185       }
1186 
1187       Map<String, Map<ServerName, List<HRegionInfo>>> assignmentsByTable =
1188         this.assignmentManager.getAssignmentsByTable();
1189 
1190       List<RegionPlan> plans = new ArrayList<RegionPlan>();
1191       for (Map<ServerName, List<HRegionInfo>> assignments : assignmentsByTable.values()) {
1192         List<RegionPlan> partialPlans = this.balancer.balanceCluster(assignments);
1193         if (partialPlans != null) plans.addAll(partialPlans);
1194       }
1195       int rpCount = 0;  // number of RegionPlans balanced so far
1196       long totalRegPlanExecTime = 0;
1197       balancerRan = plans != null;
1198       if (plans != null && !plans.isEmpty()) {
1199         for (RegionPlan plan: plans) {
1200           LOG.info("balance " + plan);
1201           long balStartTime = System.currentTimeMillis();
1202           this.assignmentManager.balance(plan);
1203           totalRegPlanExecTime += System.currentTimeMillis()-balStartTime;
1204           rpCount++;
1205           if (rpCount < plans.size() &&
1206               // if performing next balance exceeds cutoff time, exit the loop
1207               (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
1208             LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
1209               maximumBalanceTime);
1210             break;
1211           }
1212         }
1213       }
1214       if (this.cpHost != null) {
1215         try {
1216           this.cpHost.postBalance();
1217         } catch (IOException ioe) {
1218           // balancing already succeeded so don't change the result
1219           LOG.error("Error invoking master coprocessor postBalance()", ioe);
1220         }
1221       }
1222     }
1223     return balancerRan;
1224   }
1225 
1226   enum BalanceSwitchMode {
1227     SYNC,
1228     ASYNC
1229   }
1230   /**
1231    * Assigns balancer switch according to BalanceSwitchMode
1232    * @param b new balancer switch
1233    * @param mode BalanceSwitchMode
1234    * @return old balancer switch
1235    */
1236   public boolean switchBalancer(final boolean b, BalanceSwitchMode mode) {
1237     boolean oldValue = this.balanceSwitch;
1238     boolean newValue = b;
1239     try {
1240       if (this.cpHost != null) {
1241         newValue = this.cpHost.preBalanceSwitch(newValue);
1242       }
1243       if (mode == BalanceSwitchMode.SYNC) {
1244         synchronized (this.balancer) {
1245           this.balanceSwitch = newValue;
1246         }
1247       } else {
1248         this.balanceSwitch = newValue;
1249       }
1250       LOG.info("BalanceSwitch=" + newValue);
1251       if (this.cpHost != null) {
1252         this.cpHost.postBalanceSwitch(oldValue, newValue);
1253       }
1254     } catch (IOException ioe) {
1255       LOG.warn("Error flipping balance switch", ioe);
1256     }
1257     return oldValue;
1258   }
1259 
1260   @Override
1261   public boolean synchronousBalanceSwitch(final boolean b) {
1262     return switchBalancer(b, BalanceSwitchMode.SYNC);
1263   }
1264 
1265   @Override
1266   public boolean balanceSwitch(final boolean b) {
1267     return switchBalancer(b, BalanceSwitchMode.ASYNC);
1268   }
1269 
1270   /**
1271    * Switch for the background CatalogJanitor thread.
1272    * Used for testing.  The thread will continue to run.  It will just be a noop
1273    * if disabled.
1274    * @param b If false, the catalog janitor won't do anything.
1275    */
1276   public void setCatalogJanitorEnabled(final boolean b) {
1277     ((CatalogJanitor)this.catalogJanitorChore).setEnabled(b);
1278   }
1279 
1280   @Override
1281   public void move(final byte[] encodedRegionName, final byte[] destServerName)
1282   throws UnknownRegionException {
1283     Pair<HRegionInfo, ServerName> p =
1284       this.assignmentManager.getAssignment(encodedRegionName);
1285     if (p == null)
1286       throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
1287     ServerName dest = null;
1288     if (destServerName == null || destServerName.length == 0) {
1289       LOG.info("Passed destination servername is null or empty so choosing a server at random");
1290       List<ServerName> destServers = this.serverManager.getOnlineServersList();
1291       destServers.remove(p.getSecond());
1292       // If i have only one RS then destination can be null.
1293       dest = balancer.randomAssignment(destServers);
1294     } else {
1295       dest = new ServerName(Bytes.toString(destServerName));
1296     }
1297 
1298     // Now we can do the move
1299     RegionPlan rp = new RegionPlan(p.getFirst(), p.getSecond(), dest);
1300 
1301     try {
1302       checkInitialized();
1303       if (this.cpHost != null) {
1304         if (this.cpHost.preMove(p.getFirst(), p.getSecond(), dest)) {
1305           return;
1306         }
1307       }
1308       LOG.info("Added move plan " + rp + ", running balancer");
1309       this.assignmentManager.balance(rp);
1310       if (this.cpHost != null) {
1311         this.cpHost.postMove(p.getFirst(), p.getSecond(), dest);
1312       }
1313     } catch (IOException ioe) {
1314       UnknownRegionException ure = new UnknownRegionException(
1315           Bytes.toStringBinary(encodedRegionName));
1316       ure.initCause(ioe);
1317       throw ure;
1318     }
1319   }
1320 
1321   public void createTable(HTableDescriptor hTableDescriptor,
1322     byte [][] splitKeys)
1323   throws IOException {
1324     if (!isMasterRunning()) {
1325       throw new MasterNotRunningException();
1326     }
1327 
1328     HRegionInfo [] newRegions = getHRegionInfos(hTableDescriptor, splitKeys);
1329     checkInitialized();
1330     if (cpHost != null) {
1331       cpHost.preCreateTable(hTableDescriptor, newRegions);
1332     }
1333 
1334     this.executorService.submit(new CreateTableHandler(this,
1335       this.fileSystemManager, this.serverManager, hTableDescriptor, conf,
1336       newRegions, catalogTracker, assignmentManager));
1337 
1338     if (cpHost != null) {
1339       cpHost.postCreateTable(hTableDescriptor, newRegions);
1340     }
1341   }
1342 
1343   private HRegionInfo[] getHRegionInfos(HTableDescriptor hTableDescriptor,
1344     byte[][] splitKeys) {
1345     HRegionInfo[] hRegionInfos = null;
1346     if (splitKeys == null || splitKeys.length == 0) {
1347       hRegionInfos = new HRegionInfo[]{
1348           new HRegionInfo(hTableDescriptor.getName(), null, null)};
1349     } else {
1350       int numRegions = splitKeys.length + 1;
1351       hRegionInfos = new HRegionInfo[numRegions];
1352       byte[] startKey = null;
1353       byte[] endKey = null;
1354       for (int i = 0; i < numRegions; i++) {
1355         endKey = (i == splitKeys.length) ? null : splitKeys[i];
1356         hRegionInfos[i] =
1357             new HRegionInfo(hTableDescriptor.getName(), startKey, endKey);
1358         startKey = endKey;
1359       }
1360     }
1361     return hRegionInfos;
1362   }
1363 
1364   private static boolean isCatalogTable(final byte [] tableName) {
1365     return Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) ||
1366            Bytes.equals(tableName, HConstants.META_TABLE_NAME);
1367   }
1368 
1369   @Override
1370   public void deleteTable(final byte [] tableName) throws IOException {
1371     checkInitialized();
1372     if (cpHost != null) {
1373       cpHost.preDeleteTable(tableName);
1374     }
1375     this.executorService.submit(new DeleteTableHandler(tableName, this, this));
1376     if (cpHost != null) {
1377       cpHost.postDeleteTable(tableName);
1378     }
1379   }
1380 
1381   /**
1382    * Get the number of regions of the table that have been updated by the alter.
1383    *
1384    * @return Pair indicating the number of regions updated Pair.getFirst is the
1385    *         regions that are yet to be updated Pair.getSecond is the total number
1386    *         of regions of the table
1387    * @throws IOException
1388    */
1389   public Pair<Integer, Integer> getAlterStatus(byte[] tableName)
1390   throws IOException {
1391     return this.assignmentManager.getReopenStatus(tableName);
1392   }
1393 
1394   public void addColumn(byte [] tableName, HColumnDescriptor column)
1395   throws IOException {
1396     checkInitialized();
1397     if (cpHost != null) {
1398       if (cpHost.preAddColumn(tableName, column)) {
1399         return;
1400       }
1401     }
1402     new TableAddFamilyHandler(tableName, column, this, this).process();
1403     if (cpHost != null) {
1404       cpHost.postAddColumn(tableName, column);
1405     }
1406   }
1407 
1408   public void modifyColumn(byte [] tableName, HColumnDescriptor descriptor)
1409   throws IOException {
1410     checkInitialized();
1411     if (cpHost != null) {
1412       if (cpHost.preModifyColumn(tableName, descriptor)) {
1413         return;
1414       }
1415     }
1416     new TableModifyFamilyHandler(tableName, descriptor, this, this).process();
1417     if (cpHost != null) {
1418       cpHost.postModifyColumn(tableName, descriptor);
1419     }
1420   }
1421 
1422   public void deleteColumn(final byte [] tableName, final byte [] c)
1423   throws IOException {
1424     checkInitialized();
1425     if (cpHost != null) {
1426       if (cpHost.preDeleteColumn(tableName, c)) {
1427         return;
1428       }
1429     }
1430     new TableDeleteFamilyHandler(tableName, c, this, this).process();
1431     if (cpHost != null) {
1432       cpHost.postDeleteColumn(tableName, c);
1433     }
1434   }
1435 
1436   public void enableTable(final byte [] tableName) throws IOException {
1437     checkInitialized();
1438     if (cpHost != null) {
1439       cpHost.preEnableTable(tableName);
1440     }
1441     this.executorService.submit(new EnableTableHandler(this, tableName,
1442       catalogTracker, assignmentManager, false));
1443 
1444     if (cpHost != null) {
1445       cpHost.postEnableTable(tableName);
1446     }
1447   }
1448 
1449   public void disableTable(final byte [] tableName) throws IOException {
1450     checkInitialized();
1451     if (cpHost != null) {
1452       cpHost.preDisableTable(tableName);
1453     }
1454     this.executorService.submit(new DisableTableHandler(this, tableName,
1455         catalogTracker, assignmentManager, false));
1456 
1457     if (cpHost != null) {
1458       cpHost.postDisableTable(tableName);
1459     }
1460   }
1461 
1462   /**
1463    * Return the region and current deployment for the region containing
1464    * the given row. If the region cannot be found, returns null. If it
1465    * is found, but not currently deployed, the second element of the pair
1466    * may be null.
1467    */
1468   Pair<HRegionInfo, ServerName> getTableRegionForRow(
1469       final byte [] tableName, final byte [] rowKey)
1470   throws IOException {
1471     final AtomicReference<Pair<HRegionInfo, ServerName>> result =
1472       new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
1473 
1474     MetaScannerVisitor visitor =
1475       new MetaScannerVisitorBase() {
1476         @Override
1477         public boolean processRow(Result data) throws IOException {
1478           if (data == null || data.size() <= 0) {
1479             return true;
1480           }
1481           Pair<HRegionInfo, ServerName> pair = MetaReader.parseCatalogResult(data);
1482           if (pair == null) {
1483             return false;
1484           }
1485           if (!Bytes.equals(pair.getFirst().getTableName(), tableName)) {
1486             return false;
1487           }
1488           result.set(pair);
1489           return true;
1490         }
1491     };
1492 
1493     MetaScanner.metaScan(conf, visitor, tableName, rowKey, 1);
1494     return result.get();
1495   }
1496 
1497   @Override
1498   public void modifyTable(final byte[] tableName, HTableDescriptor htd)
1499       throws IOException {
1500     checkInitialized();
1501     if (cpHost != null) {
1502       cpHost.preModifyTable(tableName, htd);
1503     }
1504     TableEventHandler tblHandler = new ModifyTableHandler(tableName, htd, this, this);
1505     this.executorService.submit(tblHandler);
1506     // prevent client from querying status even before the event is being handled.
1507     tblHandler.waitForEventBeingHandled();
1508     if (cpHost != null) {
1509       cpHost.postModifyTable(tableName, htd);
1510     }
1511   }
1512 
1513   @Override
1514   public void checkTableModifiable(final byte [] tableName)
1515   throws IOException {
1516     String tableNameStr = Bytes.toString(tableName);
1517     if (isCatalogTable(tableName)) {
1518       throw new IOException("Can't modify catalog tables");
1519     }
1520     if (!MetaReader.tableExists(getCatalogTracker(), tableNameStr)) {
1521       throw new TableNotFoundException(tableNameStr);
1522     }
1523     if (!getAssignmentManager().getZKTable().
1524         isDisabledTable(Bytes.toString(tableName))) {
1525       throw new TableNotDisabledException(tableName);
1526     }
1527   }
1528 
1529   public void clearFromTransition(HRegionInfo hri) {
1530     if (this.assignmentManager.isRegionInTransition(hri) != null) {
1531       this.assignmentManager.regionOffline(hri);
1532     }
1533   }
1534 
1535   /**
1536    * @return cluster status
1537    */
1538   public ClusterStatus getClusterStatus() {
1539     // Build Set of backup masters from ZK nodes
1540     List<String> backupMasterStrings;
1541     try {
1542       backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper,
1543                               this.zooKeeper.backupMasterAddressesZNode);
1544     } catch (KeeperException e) {
1545       LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e);
1546       backupMasterStrings = new ArrayList<String>(0);
1547     }
1548     List<ServerName> backupMasters = new ArrayList<ServerName>(
1549                                           backupMasterStrings.size());
1550     for (String s: backupMasterStrings) {
1551       try {
1552         byte[] bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(this.zooKeeper.backupMasterAddressesZNode, s));
1553         if (bytes != null) {
1554           backupMasters.add(ServerName.parseVersionedServerName(bytes));
1555         }
1556       } catch (KeeperException e) {
1557         LOG.warn(this.zooKeeper.prefix("Unable to get information about " +
1558                  "backup servers"), e);
1559       }
1560     }
1561     Collections.sort(backupMasters, new Comparator<ServerName>() {
1562       public int compare(ServerName s1, ServerName s2) {
1563         return s1.getServerName().compareTo(s2.getServerName());
1564       }});
1565 
1566     return new ClusterStatus(VersionInfo.getVersion(),
1567       this.fileSystemManager.getClusterId(),
1568       this.serverManager.getOnlineServers(),
1569       this.serverManager.getDeadServers(),
1570       this.serverName,
1571       backupMasters,
1572       this.assignmentManager.getRegionsInTransition(),
1573       this.getCoprocessors());
1574   }
1575 
1576   public String getClusterId() {
1577     return (fileSystemManager == null) ? null : fileSystemManager.getClusterId();
1578   }
1579 
1580   /**
1581    * The set of loaded coprocessors is stored in a static set. Since it's
1582    * statically allocated, it does not require that HMaster's cpHost be
1583    * initialized prior to accessing it.
1584    * @return a String representation of the set of names of the loaded
1585    * coprocessors.
1586    */
1587   public static String getLoadedCoprocessors() {
1588     return CoprocessorHost.getLoadedCoprocessors().toString();
1589   }
1590 
1591   /**
1592    * @return timestamp in millis when HMaster was started.
1593    */
1594   public long getMasterStartTime() {
1595     return masterStartTime;
1596   }
1597 
1598   /**
1599    * @return timestamp in millis when HMaster became the active master.
1600    */
1601   public long getMasterActiveTime() {
1602     return masterActiveTime;
1603   }
1604 
1605   /**
1606    * @return array of coprocessor SimpleNames.
1607    */
1608   public String[] getCoprocessors() {
1609     MasterCoprocessorHost cp = getCoprocessorHost();
1610     String[] cpList = new String[0];
1611     if (cp == null) return cpList;
1612 
1613     Set<String> masterCoprocessors = cp.getCoprocessors();
1614     return masterCoprocessors.toArray(cpList);
1615   }
1616 
1617   @Override
1618   public void abort(final String msg, final Throwable t) {
1619     if (cpHost != null) {
1620       // HBASE-4014: dump a list of loaded coprocessors.
1621       LOG.fatal("Master server abort: loaded coprocessors are: " +
1622           getLoadedCoprocessors());
1623     }
1624 
1625     if (abortNow(msg, t)) {
1626       if (t != null) LOG.fatal(msg, t);
1627       else LOG.fatal(msg);
1628       this.abort = true;
1629       stop("Aborting");
1630     }
1631   }
1632 
1633   /**
1634    * We do the following in a different thread.  If it is not completed
1635    * in time, we will time it out and assume it is not easy to recover.
1636    *
1637    * 1. Create a new ZK session. (since our current one is expired)
1638    * 2. Try to become a primary master again
1639    * 3. Initialize all ZK based system trackers.
1640    * 4. Assign root and meta. (they are already assigned, but we need to update our
1641    * internal memory state to reflect it)
1642    * 5. Process any RIT if any during the process of our recovery.
1643    *
1644    * @return True if we could successfully recover from ZK session expiry.
1645    * @throws InterruptedException
1646    * @throws IOException
1647    * @throws KeeperException
1648    * @throws ExecutionException
1649    */
1650   private boolean tryRecoveringExpiredZKSession() throws InterruptedException,
1651       IOException, KeeperException, ExecutionException {
1652 
1653     this.zooKeeper.unregisterAllListeners();
1654     // add back listeners which were registered before master initialization
1655     // because they won't be added back in below Master re-initialization code
1656     if (this.registeredZKListenersBeforeRecovery != null) {
1657       for (ZooKeeperListener curListener : this.registeredZKListenersBeforeRecovery) {
1658         this.zooKeeper.registerListener(curListener);
1659       }
1660     }
1661 
1662     this.zooKeeper.reconnectAfterExpiration();
1663 
1664     Callable<Boolean> callable = new Callable<Boolean> () {
1665       public Boolean call() throws InterruptedException,
1666           IOException, KeeperException {
1667         MonitoredTask status =
1668           TaskMonitor.get().createStatus("Recovering expired ZK session");
1669         try {
1670           if (!becomeActiveMaster(status)) {
1671             return Boolean.FALSE;
1672           }
1673           serverManager.disableSSHForRoot();
1674           serverShutdownHandlerEnabled = false;
1675           initialized = false;
1676           finishInitialization(status, true);
1677           return Boolean.TRUE;
1678         } finally {
1679           status.cleanup();
1680         }
1681       }
1682     };
1683 
1684     long timeout =
1685       conf.getLong("hbase.master.zksession.recover.timeout", 300000);
1686     java.util.concurrent.ExecutorService executor =
1687       Executors.newSingleThreadExecutor();
1688     Future<Boolean> result = executor.submit(callable);
1689     executor.shutdown();
1690     if (executor.awaitTermination(timeout, TimeUnit.MILLISECONDS)
1691         && result.isDone()) {
1692       Boolean recovered = result.get();
1693       if (recovered != null) {
1694         return recovered.booleanValue();
1695       }
1696     }
1697     executor.shutdownNow();
1698     return false;
1699   }
1700 
1701   /**
1702    * Check to see if the current trigger for abort is due to ZooKeeper session
1703    * expiry, and If yes, whether we can recover from ZK session expiry.
1704    *
1705    * @param msg Original abort message
1706    * @param t   The cause for current abort request
1707    * @return true if we should proceed with abort operation, false other wise.
1708    */
1709   private boolean abortNow(final String msg, final Throwable t) {
1710     if (!this.isActiveMaster || this.stopped) {
1711       return true;
1712     }
1713 
1714     boolean failFast = conf.getBoolean("fail.fast.expired.active.master", false);
1715     if (t != null && t instanceof KeeperException.SessionExpiredException
1716         && !failFast) {
1717       try {
1718         LOG.info("Primary Master trying to recover from ZooKeeper session " +
1719             "expiry.");
1720         return !tryRecoveringExpiredZKSession();
1721       } catch (Throwable newT) {
1722         LOG.error("Primary master encountered unexpected exception while " +
1723             "trying to recover from ZooKeeper session" +
1724             " expiry. Proceeding with server abort.", newT);
1725       }
1726     }
1727     return true;
1728   }
1729 
1730   @Override
1731   public ZooKeeperWatcher getZooKeeper() {
1732     return zooKeeper;
1733   }
1734 
1735   @Override
1736   public MasterCoprocessorHost getCoprocessorHost() {
1737     return cpHost;
1738   }
1739 
1740   @Override
1741   public ServerName getServerName() {
1742     return this.serverName;
1743   }
1744 
1745   @Override
1746   public CatalogTracker getCatalogTracker() {
1747     return catalogTracker;
1748   }
1749 
1750   @Override
1751   public AssignmentManager getAssignmentManager() {
1752     return this.assignmentManager;
1753   }
1754 
1755   public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
1756     return rsFatals;
1757   }
1758 
1759   @SuppressWarnings("deprecation")
1760   @Override
1761   public void shutdown() {
1762     if (cpHost != null) {
1763       try {
1764         cpHost.preShutdown();
1765       } catch (IOException ioe) {
1766         LOG.error("Error call master coprocessor preShutdown()", ioe);
1767       }
1768     }
1769     if (mxBean != null) {
1770       MBeanUtil.unregisterMBean(mxBean);
1771       mxBean = null;
1772     }
1773     if (this.assignmentManager != null) this.assignmentManager.shutdown();
1774     if (this.serverManager != null) this.serverManager.shutdownCluster();
1775 
1776     try {
1777       if (this.clusterStatusTracker != null){
1778         this.clusterStatusTracker.setClusterDown();
1779       }
1780     } catch (KeeperException e) {
1781       if (e instanceof KeeperException.SessionExpiredException) {
1782         LOG.warn("ZK session expired. Retry a new connection...");
1783         try {
1784           this.zooKeeper.reconnectAfterExpiration();
1785           this.clusterStatusTracker.setClusterDown();
1786         } catch (Exception ex) {
1787           LOG.error("Retry setClusterDown failed", ex);
1788         }
1789       } else {
1790         LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
1791       }
1792     }
1793   }
1794 
1795   @Override
1796   public void stopMaster() {
1797     if (cpHost != null) {
1798       try {
1799         cpHost.preStopMaster();
1800       } catch (IOException ioe) {
1801         LOG.error("Error call master coprocessor preStopMaster()", ioe);
1802       }
1803     }
1804     stop("Stopped by " + Thread.currentThread().getName());
1805   }
1806 
1807   @Override
1808   public void stop(final String why) {
1809     LOG.info(why);
1810     this.stopped = true;
1811     // We wake up the stopSleeper to stop immediately
1812     stopSleeper.skipSleepCycle();
1813     // If we are a backup master, we need to interrupt wait
1814     if (this.activeMasterManager != null) {
1815       synchronized (this.activeMasterManager.clusterHasActiveMaster) {
1816         this.activeMasterManager.clusterHasActiveMaster.notifyAll();
1817       }
1818     }
1819     // If no region server is online then master may stuck waiting on -ROOT- and .META. to come on
1820     // line. See HBASE-8422.
1821     if (this.catalogTracker != null && this.serverManager.getOnlineServers().isEmpty()) {
1822       this.catalogTracker.stop();
1823     }
1824   }
1825 
1826   @Override
1827   public boolean isStopped() {
1828     return this.stopped;
1829   }
1830 
1831   public boolean isAborted() {
1832     return this.abort;
1833   }
1834 
1835   void checkInitialized() throws PleaseHoldException {
1836     if (!this.initialized) {
1837       throw new PleaseHoldException("Master is initializing");
1838     }
1839   }
1840 
1841   /**
1842    * Report whether this master is currently the active master or not.
1843    * If not active master, we are parked on ZK waiting to become active.
1844    *
1845    * This method is used for testing.
1846    *
1847    * @return true if active master, false if not.
1848    */
1849   public boolean isActiveMaster() {
1850     return isActiveMaster;
1851   }
1852 
1853   /**
1854    * Report whether this master has completed with its initialization and is
1855    * ready.  If ready, the master is also the active master.  A standby master
1856    * is never ready.
1857    *
1858    * This method is used for testing.
1859    *
1860    * @return true if master is ready to go, false if not.
1861    */
1862   public boolean isInitialized() {
1863     return initialized;
1864   }
1865 
1866   /**
1867    * ServerShutdownHandlerEnabled is set false before completing
1868    * assignRootAndMeta to prevent processing of ServerShutdownHandler.
1869    * @return true if assignRootAndMeta has completed;
1870    */
1871   public boolean isServerShutdownHandlerEnabled() {
1872     return this.serverShutdownHandlerEnabled;
1873   }
1874 
1875   public boolean shouldSplitMetaSeparately() {
1876     return this.shouldSplitMetaSeparately;
1877   }
1878 
1879   /**
1880    * Report whether this master has started initialization and is about to do meta region assignment
1881    * @return true if master is in initialization & about to assign ROOT & META regions
1882    */
1883   public boolean isInitializationStartsMetaRegoinAssignment() {
1884     return this.initializationBeforeMetaAssignment;
1885   }
1886 
1887   @Override
1888   @Deprecated
1889   public void assign(final byte[] regionName, final boolean force)
1890       throws IOException {
1891     assign(regionName);
1892   }
1893 
1894   @Override
1895   public void assign(final byte [] regionName)throws IOException {
1896     checkInitialized();
1897     Pair<HRegionInfo, ServerName> pair =
1898       MetaReader.getRegion(this.catalogTracker, regionName);
1899     if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
1900     if (cpHost != null) {
1901       if (cpHost.preAssign(pair.getFirst())) {
1902         return;
1903       }
1904     }
1905     assignRegion(pair.getFirst());
1906     if (cpHost != null) {
1907       cpHost.postAssign(pair.getFirst());
1908     }
1909   }
1910 
1911 
1912 
1913   public void assignRegion(HRegionInfo hri) {
1914     assignmentManager.assign(hri, true);
1915   }
1916 
1917   @Override
1918   public void unassign(final byte [] regionName, final boolean force)
1919   throws IOException {
1920     checkInitialized();
1921     Pair<HRegionInfo, ServerName> pair =
1922       MetaReader.getRegion(this.catalogTracker, regionName);
1923     if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
1924     HRegionInfo hri = pair.getFirst();
1925     if (cpHost != null) {
1926       if (cpHost.preUnassign(hri, force)) {
1927         return;
1928       }
1929     }
1930     if (force) {
1931       this.assignmentManager.regionOffline(hri);
1932       assignRegion(hri);
1933     } else {
1934       this.assignmentManager.unassign(hri, force);
1935     }
1936     if (cpHost != null) {
1937       cpHost.postUnassign(hri, force);
1938     }
1939   }
1940 
1941   /**
1942    * Get HTD array for given tables
1943    * @param tableNames
1944    * @return HTableDescriptor[]
1945    */
1946   public HTableDescriptor[] getHTableDescriptors(List<String> tableNames)
1947       throws IOException {
1948     List<HTableDescriptor> descriptors =
1949       new ArrayList<HTableDescriptor>(tableNames.size());
1950     
1951     boolean bypass = false;
1952     if (this.cpHost != null) {
1953       bypass = this.cpHost.preGetTableDescriptors(tableNames, descriptors);
1954     }
1955 
1956     if (!bypass) {
1957       for (String s: tableNames) {
1958         HTableDescriptor htd = null;
1959         try {
1960           htd = this.tableDescriptors.get(s);
1961         } catch (IOException e) {
1962           LOG.warn("Failed getting descriptor for " + s, e);
1963         }
1964         if (htd == null) continue;
1965         descriptors.add(htd);
1966       }
1967     }
1968 
1969     if (this.cpHost != null) {
1970       this.cpHost.postGetTableDescriptors(descriptors);
1971     }
1972 
1973     return descriptors.toArray(new HTableDescriptor [] {});
1974   }
1975 
1976   @Override
1977   public <T extends CoprocessorProtocol> boolean registerProtocol(
1978       Class<T> protocol, T handler) {
1979 
1980     /* No stacking of protocol handlers is currently allowed.  The
1981      * first to claim wins!
1982      */
1983     if (protocolHandlers.containsKey(protocol)) {
1984       LOG.error("Protocol "+protocol.getName()+
1985           " already registered, rejecting request from "+
1986           handler
1987       );
1988       return false;
1989     }
1990 
1991     protocolHandlers.putInstance(protocol, handler);
1992     protocolHandlerNames.put(protocol.getName(), protocol);
1993     if (LOG.isDebugEnabled()) {
1994       LOG.debug("Registered master protocol handler: protocol="+protocol.getName());
1995     }
1996     return true;
1997   }
1998 
1999   @Override
2000   public ExecResult execCoprocessor(Exec call) throws IOException {
2001     Class<? extends CoprocessorProtocol> protocol = call.getProtocol();
2002     if (protocol == null) {
2003       String protocolName = call.getProtocolName();
2004       if (LOG.isTraceEnabled()) {
2005         LOG.trace("Received dynamic protocol exec call with protocolName " + protocolName);
2006       }
2007       // detect the actual protocol class
2008       protocol  = protocolHandlerNames.get(protocolName);
2009       if (protocol == null) {
2010         throw new HBaseRPC.UnknownProtocolException(protocol,
2011             "No matching handler for master protocol "+protocolName);
2012       }
2013     }
2014     if (!protocolHandlers.containsKey(protocol)) {
2015       throw new HBaseRPC.UnknownProtocolException(protocol,
2016           "No matching handler for protocol ");
2017     }
2018 
2019     CoprocessorProtocol handler = protocolHandlers.getInstance(protocol);
2020     Object value;
2021 
2022     try {
2023       Method method = protocol.getMethod(
2024           call.getMethodName(), call.getParameterClasses());
2025       method.setAccessible(true);
2026 
2027       value = method.invoke(handler, call.getParameters());
2028     } catch (InvocationTargetException e) {
2029       Throwable target = e.getTargetException();
2030       if (target instanceof IOException) {
2031         throw (IOException)target;
2032       }
2033       IOException ioe = new IOException(target.toString());
2034       ioe.setStackTrace(target.getStackTrace());
2035       throw ioe;
2036     } catch (Throwable e) {
2037       if (!(e instanceof IOException)) {
2038         LOG.error("Unexpected throwable object ", e);
2039       }
2040       IOException ioe = new IOException(e.toString());
2041       ioe.setStackTrace(e.getStackTrace());
2042       throw ioe;
2043     }
2044 
2045     return new ExecResult(value);
2046   }
2047 
2048   /**
2049    * Get all table descriptors
2050    * @return All descriptors or null if none.
2051    * @throws IOException
2052    */
2053   public HTableDescriptor [] getHTableDescriptors() throws IOException {
2054     List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
2055     boolean bypass = false;
2056     if (this.cpHost != null) {
2057       bypass = this.cpHost.preGetTableDescriptors(null, descriptors);
2058     }
2059     if (!bypass) {
2060       descriptors.addAll(this.tableDescriptors.getAll().values());
2061     }
2062     if (this.cpHost != null) {
2063       this.cpHost.postGetTableDescriptors(descriptors);
2064     }
2065     return descriptors.toArray(new HTableDescriptor [] {});
2066   }
2067 
2068   /**
2069    * Compute the average load across all region servers.
2070    * Currently, this uses a very naive computation - just uses the number of
2071    * regions being served, ignoring stats about number of requests.
2072    * @return the average load
2073    */
2074   public double getAverageLoad() {
2075     return this.assignmentManager.getAverageLoad();
2076   }
2077 
2078   /**
2079    * Special method, only used by hbck.
2080    */
2081   @Override
2082   public void offline(final byte[] regionName) throws IOException {
2083     Pair<HRegionInfo, ServerName> pair =
2084       MetaReader.getRegion(this.catalogTracker, regionName);
2085     if (pair == null) throw new UnknownRegionException(Bytes.toStringBinary(regionName));
2086     HRegionInfo hri = pair.getFirst();
2087     this.assignmentManager.regionOffline(hri);
2088   }
2089 
2090   /**
2091    * Utility for constructing an instance of the passed HMaster class.
2092    * @param masterClass
2093    * @param conf
2094    * @return HMaster instance.
2095    */
2096   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
2097       final Configuration conf)  {
2098     try {
2099       Constructor<? extends HMaster> c =
2100         masterClass.getConstructor(Configuration.class);
2101       return c.newInstance(conf);
2102     } catch (InvocationTargetException ite) {
2103       Throwable target = ite.getTargetException() != null?
2104         ite.getTargetException(): ite;
2105       if (target.getCause() != null) target = target.getCause();
2106       throw new RuntimeException("Failed construction of Master: " +
2107         masterClass.toString(), target);
2108     } catch (Exception e) {
2109       throw new RuntimeException("Failed construction of Master: " +
2110         masterClass.toString() + ((e.getCause() != null)?
2111           e.getCause().getMessage(): ""), e);
2112     }
2113   }
2114 
2115   /**
2116    * @see org.apache.hadoop.hbase.master.HMasterCommandLine
2117    */
2118   public static void main(String [] args) throws Exception {
2119 	VersionInfo.logVersion();
2120     new HMasterCommandLine(HMaster.class).doMain(args);
2121   }
2122 
2123   /**
2124    * Register bean with platform management server
2125    */
2126   @SuppressWarnings("deprecation")
2127   void registerMBean() {
2128     MXBeanImpl mxBeanInfo = MXBeanImpl.init(this);
2129     MBeanUtil.registerMBean("Master", "Master", mxBeanInfo);
2130     LOG.info("Registered HMaster MXBean");
2131   }
2132 
2133   /**
2134    * Exposed for Testing!
2135    * @return the current hfile cleaner
2136    */
2137   public HFileCleaner getHFileCleaner() {
2138     return this.hfileCleaner;
2139   }
2140 
2141   private boolean isHealthCheckerConfigured() {
2142     String healthScriptLocation = this.conf.get(HConstants.HEALTH_SCRIPT_LOC);
2143     return org.apache.commons.lang.StringUtils.isNotBlank(healthScriptLocation);
2144   }
2145 
2146   /**
2147    * Exposed for TESTING!
2148    * @return the underlying snapshot manager
2149    */
2150   public SnapshotManager getSnapshotManagerForTesting() {
2151     return this.snapshotManager;
2152    }
2153 
2154 
2155   /**
2156    * Triggers an asynchronous attempt to take a snapshot.
2157    * {@inheritDoc}
2158    */
2159   @Override
2160   public long snapshot(final HSnapshotDescription request) throws IOException {
2161     LOG.debug("Submitting snapshot request for:" +
2162         SnapshotDescriptionUtils.toString(request.getProto()));
2163     try {
2164       this.snapshotManager.checkSnapshotSupport();
2165     } catch (UnsupportedOperationException e) {
2166       throw new IOException(e);
2167     }
2168 
2169     // get the snapshot information
2170     SnapshotDescription snapshot = SnapshotDescriptionUtils.validate(request.getProto(),
2171       this.conf);
2172 
2173     snapshotManager.takeSnapshot(snapshot);
2174 
2175     // send back the max amount of time the client should wait for the snapshot to complete
2176     long waitTime = SnapshotDescriptionUtils.getMaxMasterTimeout(conf, snapshot.getType(),
2177       SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME);
2178     return waitTime;
2179   }
2180 
2181   /**
2182    * List the currently available/stored snapshots. Any in-progress snapshots are ignored
2183    */
2184   @Override
2185   public List<HSnapshotDescription> getCompletedSnapshots() throws IOException {
2186     List<HSnapshotDescription> availableSnapshots = new ArrayList<HSnapshotDescription>();
2187     List<SnapshotDescription> snapshots = snapshotManager.getCompletedSnapshots();
2188 
2189     // convert to writables
2190     for (SnapshotDescription snapshot: snapshots) {
2191       availableSnapshots.add(new HSnapshotDescription(snapshot));
2192     }
2193 
2194     return availableSnapshots;
2195   }
2196 
2197   /**
2198    * Execute Delete Snapshot operation.
2199    * @throws ServiceException wrapping SnapshotDoesNotExistException if specified snapshot did not
2200    * exist.
2201    */
2202   @Override
2203   public void deleteSnapshot(final HSnapshotDescription request) throws IOException {
2204     try {
2205       this.snapshotManager.checkSnapshotSupport();
2206     } catch (UnsupportedOperationException e) {
2207       throw new IOException(e);
2208     }
2209 
2210     snapshotManager.deleteSnapshot(request.getProto());
2211   }
2212 
2213   /**
2214    * Checks if the specified snapshot is done.
2215    * @return true if the snapshot is in file system ready to use,
2216    * false if the snapshot is in the process of completing
2217    * @throws ServiceException wrapping UnknownSnapshotException if invalid snapshot, or
2218    * a wrapped HBaseSnapshotException with progress failure reason.
2219    */
2220   @Override
2221   public boolean isSnapshotDone(final HSnapshotDescription request) throws IOException {
2222     LOG.debug("Checking to see if snapshot from request:" +
2223       SnapshotDescriptionUtils.toString(request.getProto()) + " is done");
2224     return snapshotManager.isSnapshotDone(request.getProto());
2225   }
2226 
2227   /**
2228    * Execute Restore/Clone snapshot operation.
2229    *
2230    * <p>If the specified table exists a "Restore" is executed, replacing the table
2231    * schema and directory data with the content of the snapshot.
2232    * The table must be disabled, or a UnsupportedOperationException will be thrown.
2233    *
2234    * <p>If the table doesn't exist a "Clone" is executed, a new table is created
2235    * using the schema at the time of the snapshot, and the content of the snapshot.
2236    *
2237    * <p>The restore/clone operation does not require copying HFiles. Since HFiles
2238    * are immutable the table can point to and use the same files as the original one.
2239    */
2240   @Override
2241   public void restoreSnapshot(final HSnapshotDescription request) throws IOException {
2242     try {
2243       this.snapshotManager.checkSnapshotSupport();
2244     } catch (UnsupportedOperationException e) {
2245       throw new IOException(e);
2246     }
2247 
2248     snapshotManager.restoreSnapshot(request.getProto());
2249   }
2250 
2251   /**
2252    * Returns the status of the requested snapshot restore/clone operation.
2253    * This method is not exposed to the user, it is just used internally by HBaseAdmin
2254    * to verify if the restore is completed.
2255    *
2256    * No exceptions are thrown if the restore is not running, the result will be "done".
2257    *
2258    * @return done <tt>true</tt> if the restore/clone operation is completed.
2259    * @throws RestoreSnapshotExcepton if the operation failed.
2260    */
2261   @Override
2262   public boolean isRestoreSnapshotDone(final HSnapshotDescription request) throws IOException {
2263     return snapshotManager.isRestoreDone(request.getProto());
2264   }
2265 
2266   /**
2267    * Return all table names.
2268    * @return the list of table names
2269    * @throws IOException if an error occurred while getting the list of tables
2270    */
2271   @Override
2272   public String[] getTableNames() throws IOException {
2273     // Anyone is allowed to see the names of tables, so there is no coprocessor
2274     // hook nor AccessController interception necessary
2275     Collection<HTableDescriptor> descriptors = tableDescriptors.getAll().values();
2276     Iterator<HTableDescriptor> iter = descriptors.iterator();
2277     String names[] = new String[descriptors.size()];
2278     int i = 0;
2279     while (iter.hasNext()) {
2280       names[i++] = iter.next().getNameAsString();
2281     }
2282     return names;
2283   }
2284 }