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