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