View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import java.io.IOException;
23  import java.lang.reflect.Constructor;
24  import java.lang.reflect.InvocationTargetException;
25  import java.net.InetSocketAddress;
26  import java.net.UnknownHostException;
27  import java.util.ArrayList;
28  import java.util.Arrays;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.concurrent.atomic.AtomicReference;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.Chore;
37  import org.apache.hadoop.hbase.ClusterStatus;
38  import org.apache.hadoop.hbase.HColumnDescriptor;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.HMsg;
41  import org.apache.hadoop.hbase.HRegionInfo;
42  import org.apache.hadoop.hbase.HServerAddress;
43  import org.apache.hadoop.hbase.HServerInfo;
44  import org.apache.hadoop.hbase.HTableDescriptor;
45  import org.apache.hadoop.hbase.MasterNotRunningException;
46  import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
47  import org.apache.hadoop.hbase.Server;
48  import org.apache.hadoop.hbase.TableExistsException;
49  import org.apache.hadoop.hbase.TableNotDisabledException;
50  import org.apache.hadoop.hbase.TableNotFoundException;
51  import org.apache.hadoop.hbase.UnknownRegionException;
52  import org.apache.hadoop.hbase.catalog.CatalogTracker;
53  import org.apache.hadoop.hbase.catalog.MetaEditor;
54  import org.apache.hadoop.hbase.catalog.MetaReader;
55  import org.apache.hadoop.hbase.client.HConnection;
56  import org.apache.hadoop.hbase.client.HConnectionManager;
57  import org.apache.hadoop.hbase.client.MetaScanner;
58  import org.apache.hadoop.hbase.client.Result;
59  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
60  import org.apache.hadoop.hbase.executor.ExecutorService;
61  import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
62  import org.apache.hadoop.hbase.ipc.HBaseRPC;
63  import org.apache.hadoop.hbase.ipc.HBaseRPCProtocolVersion;
64  import org.apache.hadoop.hbase.ipc.HBaseServer;
65  import org.apache.hadoop.hbase.ipc.HMasterInterface;
66  import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
67  import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
68  import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
69  import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
70  import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
71  import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
72  import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
73  import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
74  import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
75  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
76  import org.apache.hadoop.hbase.regionserver.HRegion;
77  import org.apache.hadoop.hbase.replication.regionserver.Replication;
78  import org.apache.hadoop.hbase.security.User;
79  import org.apache.hadoop.hbase.util.Bytes;
80  import org.apache.hadoop.hbase.util.InfoServer;
81  import org.apache.hadoop.hbase.util.Pair;
82  import org.apache.hadoop.hbase.util.Sleeper;
83  import org.apache.hadoop.hbase.util.Threads;
84  import org.apache.hadoop.hbase.util.VersionInfo;
85  import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
86  import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
87  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
88  import org.apache.hadoop.io.MapWritable;
89  import org.apache.hadoop.io.Text;
90  import org.apache.hadoop.ipc.RemoteException;
91  import org.apache.hadoop.net.DNS;
92  import org.apache.zookeeper.KeeperException;
93  import org.apache.zookeeper.Watcher;
94  
95  /**
96   * HMaster is the "master server" for HBase. An HBase cluster has one active
97   * master.  If many masters are started, all compete.  Whichever wins goes on to
98   * run the cluster.  All others park themselves in their constructor until
99   * master or cluster shutdown or until the active master loses its lease in
100  * zookeeper.  Thereafter, all running master jostle to take over master role.
101  *
102  * <p>The Master can be asked shutdown the cluster. See {@link #shutdown()}.  In
103  * this case it will tell all regionservers to go down and then wait on them
104  * all reporting in that they are down.  This master will then shut itself down.
105  *
106  * <p>You can also shutdown just this master.  Call {@link #stopMaster()}.
107  *
108  * @see HMasterInterface
109  * @see HMasterRegionInterface
110  * @see Watcher
111  */
112 public class HMaster extends Thread
113 implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
114   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
115 
116   // MASTER is name of the webapp and the attribute name used stuffing this
117   //instance into web context.
118   public static final String MASTER = "master";
119 
120   // The configuration for the Master
121   private final Configuration conf;
122   // server for the web ui
123   private InfoServer infoServer;
124 
125   // Our zk client.
126   private ZooKeeperWatcher zooKeeper;
127   // Manager and zk listener for master election
128   private ActiveMasterManager activeMasterManager;
129   // Region server tracker
130   private RegionServerTracker regionServerTracker;
131 
132   // RPC server for the HMaster
133   private final HBaseServer rpcServer;
134   // Address of the HMaster
135   private final HServerAddress address;
136   // Metrics for the HMaster
137   private final MasterMetrics metrics;
138   // file system manager for the master FS operations
139   private MasterFileSystem fileSystemManager;
140 
141   private HConnection connection;
142 
143   // server manager to deal with region server info
144   private ServerManager serverManager;
145 
146   // manager of assignment nodes in zookeeper
147   AssignmentManager assignmentManager;
148   // manager of catalog regions
149   private CatalogTracker catalogTracker;
150   // Cluster status zk tracker and local setter
151   private ClusterStatusTracker clusterStatusTracker;
152 
153   // This flag is for stopping this Master instance.  Its set when we are
154   // stopping or aborting
155   private volatile boolean stopped = false;
156   // Set on abort -- usually failure of our zk session.
157   private volatile boolean abort = false;
158   // flag set after we become the active master (used for testing)
159   private volatile boolean isActiveMaster = false;
160   // flag set after we complete initialization once active (used for testing)
161   private volatile boolean initialized = false;
162 
163   // Instance of the hbase executor service.
164   ExecutorService executorService;
165 
166   private LoadBalancer balancer;
167   private Thread balancerChore;
168   // If 'true', the balancer is 'on'.  If 'false', the balancer will not run.
169   private volatile boolean balanceSwitch = true;
170 
171   private Thread catalogJanitorChore;
172   private LogCleaner logCleaner;
173 
174   /**
175    * Initializes the HMaster. The steps are as follows:
176    * <p>
177    * <ol>
178    * <li>Initialize HMaster RPC and address
179    * <li>Connect to ZooKeeper.
180    * </ol>
181    * <p>
182    * Remaining steps of initialization occur in {@link #run()} so that they
183    * run in their own thread rather than within the context of the constructor.
184    * @throws InterruptedException
185    */
186   public HMaster(final Configuration conf)
187   throws IOException, KeeperException, InterruptedException {
188     this.conf = conf;
189 
190     /*
191      * Determine address and initialize RPC server (but do not start).
192      * The RPC server ports can be ephemeral. Create a ZKW instance.
193      */
194     HServerAddress a = new HServerAddress(getMyAddress(this.conf));
195     int numHandlers = conf.getInt("hbase.regionserver.handler.count", 10);
196     this.rpcServer = HBaseRPC.getServer(this,
197       new Class<?>[]{HMasterInterface.class, HMasterRegionInterface.class},
198       a.getBindAddress(), a.getPort(),
199       numHandlers,
200       0, // we dont use high priority handlers in master
201       false, conf,
202       0); // this is a DNC w/o high priority handlers
203     this.address = new HServerAddress(rpcServer.getListenerAddress());
204 
205     // initialize server principal (if using secure Hadoop)
206     User.login(conf, "hbase.master.keytab.file",
207         "hbase.master.kerberos.principal", this.address.getHostname());
208 
209     // set the thread name now we have an address
210     setName(MASTER + "-" + this.address);
211 
212     Replication.decorateMasterConfiguration(this.conf);
213 
214     this.rpcServer.startThreads();
215 
216     // Hack! Maps DFSClient => Master for logs.  HDFS made this
217     // config param for task trackers, but we can piggyback off of it.
218     if (this.conf.get("mapred.task.id") == null) {
219       this.conf.set("mapred.task.id", "hb_m_" + this.address.toString() +
220         "_" + System.currentTimeMillis());
221     }
222 
223     this.zooKeeper = new ZooKeeperWatcher(conf, MASTER + ":" +
224         address.getPort(), this);
225 
226     this.metrics = new MasterMetrics(getServerName());
227   }
228 
229   /**
230    * Stall startup if we are designated a backup master; i.e. we want someone
231    * else to become the master before proceeding.
232    * @param c
233    * @param amm
234    * @throws InterruptedException
235    */
236   private static void stallIfBackupMaster(final Configuration c,
237       final ActiveMasterManager amm)
238   throws InterruptedException {
239     // If we're a backup master, stall until a primary to writes his address
240     if (!c.getBoolean(HConstants.MASTER_TYPE_BACKUP,
241       HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
242       return;
243     }
244     LOG.debug("HMaster started in backup mode.  " +
245       "Stalling until master znode is written.");
246     // This will only be a minute or so while the cluster starts up,
247     // so don't worry about setting watches on the parent znode
248     while (!amm.isActiveMaster()) {
249       LOG.debug("Waiting for master address ZNode to be written " +
250         "(Also watching cluster state node)");
251       Thread.sleep(c.getInt("zookeeper.session.timeout", 180 * 1000));
252     }
253   }
254 
255   /**
256    * Main processing loop for the HMaster.
257    * <ol>
258    * <li>Block until becoming active master
259    * <li>Finish initialization via {@link #finishInitialization()}
260    * <li>Enter loop until we are stopped
261    * <li>Stop services and perform cleanup once stopped
262    * </ol>
263    */
264   @Override
265   public void run() {
266     try {
267       /*
268        * Block on becoming the active master.
269        *
270        * We race with other masters to write our address into ZooKeeper.  If we
271        * succeed, we are the primary/active master and finish initialization.
272        *
273        * If we do not succeed, there is another active master and we should
274        * now wait until it dies to try and become the next active master.  If we
275        * do not succeed on our first attempt, this is no longer a cluster startup.
276        */
277       this.activeMasterManager = new ActiveMasterManager(zooKeeper, address, this);
278       this.zooKeeper.registerListener(activeMasterManager);
279       stallIfBackupMaster(this.conf, this.activeMasterManager);
280       this.activeMasterManager.blockUntilBecomingActiveMaster();
281       // We are either the active master or we were asked to shutdown
282       if (!this.stopped) {
283         finishInitialization();
284         loop();
285       }
286     } catch (Throwable t) {
287       abort("Unhandled exception. Starting shutdown.", t);
288     } finally {
289       stopChores();
290       // Wait for all the remaining region servers to report in IFF we were
291       // running a cluster shutdown AND we were NOT aborting.
292       if (!this.abort && this.serverManager != null &&
293           this.serverManager.isClusterShutdown()) {
294         this.serverManager.letRegionServersShutdown();
295       }
296       stopServiceThreads();
297       // Stop services started for both backup and active masters
298       if (this.activeMasterManager != null) this.activeMasterManager.stop();
299       if (this.catalogTracker != null) this.catalogTracker.stop();
300       if (this.serverManager != null) this.serverManager.stop();
301       if (this.assignmentManager != null) this.assignmentManager.stop();
302       HConnectionManager.deleteConnection(this.conf, true);
303       this.zooKeeper.close();
304     }
305     LOG.info("HMaster main thread exiting");
306   }
307 
308   private void loop() {
309     // Check if we should stop every second.
310     Sleeper sleeper = new Sleeper(1000, this);
311     while (!this.stopped) {
312       sleeper.sleep();
313     }
314   }
315 
316   /**
317    * Finish initialization of HMaster after becoming the primary master.
318    *
319    * <ol>
320    * <li>Initialize master components - file system manager, server manager,
321    *     assignment manager, region server tracker, catalog tracker, etc</li>
322    * <li>Start necessary service threads - rpc server, info server,
323    *     executor services, etc</li>
324    * <li>Set cluster as UP in ZooKeeper</li>
325    * <li>Wait for RegionServers to check-in</li>
326    * <li>Split logs and perform data recovery, if necessary</li>
327    * <li>Ensure assignment of root and meta regions<li>
328    * <li>Handle either fresh cluster start or master failover</li>
329    * </ol>
330    *
331    * @throws IOException
332    * @throws InterruptedException
333    * @throws KeeperException
334    */
335   private void finishInitialization()
336   throws IOException, InterruptedException, KeeperException {
337 
338     isActiveMaster = true;
339 
340     /*
341      * We are active master now... go initialize components we need to run.
342      * Note, there may be dross in zk from previous runs; it'll get addressed
343      * below after we determine if cluster startup or failover.
344      */
345 
346     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
347     this.fileSystemManager = new MasterFileSystem(this, metrics);
348     this.connection = HConnectionManager.getConnection(conf);
349     this.executorService = new ExecutorService(getServerName());
350 
351     this.serverManager = new ServerManager(this, this, metrics);
352 
353     this.catalogTracker = new CatalogTracker(this.zooKeeper, this.connection,
354       this, conf.getInt("hbase.master.catalog.timeout", Integer.MAX_VALUE));
355     this.catalogTracker.start();
356 
357     this.assignmentManager = new AssignmentManager(this, serverManager,
358       this.catalogTracker, this.executorService);
359     this.balancer = new LoadBalancer(conf);
360     zooKeeper.registerListenerFirst(assignmentManager);
361 
362     this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
363       this.serverManager);
364     this.regionServerTracker.start();
365 
366     // Set the cluster as up.  If new RSs, they'll be waiting on this before
367     // going ahead with their startup.
368     this.clusterStatusTracker = new ClusterStatusTracker(getZooKeeper(), this);
369     this.clusterStatusTracker.start();
370     boolean wasUp = this.clusterStatusTracker.isClusterUp();
371     if (!wasUp) this.clusterStatusTracker.setClusterUp();
372 
373     LOG.info("Server active/primary master; " + this.address +
374       ", sessionid=0x" +
375       Long.toHexString(this.zooKeeper.getZooKeeper().getSessionId()) +
376       ", cluster-up flag was=" + wasUp);
377 
378     // start up all service threads.
379     startServiceThreads();
380 
381     // Wait for region servers to report in.  Returns count of regions.
382     int regionCount = this.serverManager.waitForRegionServers();
383 
384     // TODO: Should do this in background rather than block master startup
385     this.fileSystemManager.
386       splitLogAfterStartup(this.serverManager.getOnlineServers());
387 
388     // Make sure root and meta assigned before proceeding.
389     assignRootAndMeta();
390 
391     // Is this fresh start with no regions assigned or are we a master joining
392     // an already-running cluster?  If regionsCount == 0, then for sure a
393     // fresh start.  TOOD: Be fancier.  If regionsCount == 2, perhaps the
394     // 2 are .META. and -ROOT- and we should fall into the fresh startup
395     // branch below.  For now, do processFailover.
396     if (regionCount == 0) {
397       LOG.info("Master startup proceeding: cluster startup");
398       this.assignmentManager.cleanoutUnassigned();
399       this.assignmentManager.assignAllUserRegions();
400     } else {
401       LOG.info("Master startup proceeding: master failover");
402       this.assignmentManager.processFailover();
403     }
404 
405     // Start balancer and meta catalog janitor after meta and regions have
406     // been assigned.
407     this.balancerChore = getAndStartBalancerChore(this);
408     this.catalogJanitorChore =
409       Threads.setDaemonThreadRunning(new CatalogJanitor(this, this));
410 
411     LOG.info("Master has completed initialization");
412     initialized = true;
413   }
414 
415   /**
416    * Check <code>-ROOT-</code> and <code>.META.</code> are assigned.  If not,
417    * assign them.
418    * @throws InterruptedException
419    * @throws IOException
420    * @throws KeeperException
421    * @return Count of regions we assigned.
422    */
423   int assignRootAndMeta()
424   throws InterruptedException, IOException, KeeperException {
425     int assigned = 0;
426     long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
427 
428     // Work on ROOT region.  Is it in zk in transition?
429     boolean rit = this.assignmentManager.
430       processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.ROOT_REGIONINFO);
431     if (!catalogTracker.verifyRootRegionLocation(timeout)) {
432       this.assignmentManager.assignRoot();
433       this.catalogTracker.waitForRoot();
434       assigned++;
435     }
436     LOG.info("-ROOT- assigned=" + assigned + ", rit=" + rit +
437       ", location=" + catalogTracker.getRootLocation());
438 
439     // Work on meta region
440     rit = this.assignmentManager.
441       processRegionInTransitionAndBlockUntilAssigned(HRegionInfo.FIRST_META_REGIONINFO);
442     if (!this.catalogTracker.verifyMetaRegionLocation(timeout)) {
443       this.assignmentManager.assignMeta();
444       this.catalogTracker.waitForMeta();
445       // Above check waits for general meta availability but this does not
446       // guarantee that the transition has completed
447       this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
448       assigned++;
449     }
450     LOG.info(".META. assigned=" + assigned + ", rit=" + rit +
451       ", location=" + catalogTracker.getMetaLocation());
452     return assigned;
453   }
454 
455   /*
456    * @return This masters' address.
457    * @throws UnknownHostException
458    */
459   private static String getMyAddress(final Configuration c)
460   throws UnknownHostException {
461     // Find out our address up in DNS.
462     String s = DNS.getDefaultHost(c.get("hbase.master.dns.interface","default"),
463       c.get("hbase.master.dns.nameserver","default"));
464     s += ":" + c.get(HConstants.MASTER_PORT,
465         Integer.toString(HConstants.DEFAULT_MASTER_PORT));
466     return s;
467   }
468 
469   /** @return HServerAddress of the master server */
470   public HServerAddress getMasterAddress() {
471     return this.address;
472   }
473 
474   public long getProtocolVersion(String protocol, long clientVersion) {
475     return HBaseRPCProtocolVersion.versionID;
476   }
477 
478   /** @return InfoServer object. Maybe null.*/
479   public InfoServer getInfoServer() {
480     return this.infoServer;
481   }
482 
483   @Override
484   public Configuration getConfiguration() {
485     return this.conf;
486   }
487 
488   @Override
489   public ServerManager getServerManager() {
490     return this.serverManager;
491   }
492 
493   @Override
494   public ExecutorService getExecutorService() {
495     return this.executorService;
496   }
497 
498   @Override
499   public MasterFileSystem getMasterFileSystem() {
500     return this.fileSystemManager;
501   }
502 
503   /**
504    * Get the ZK wrapper object - needed by master_jsp.java
505    * @return the zookeeper wrapper
506    */
507   public ZooKeeperWatcher getZooKeeperWatcher() {
508     return this.zooKeeper;
509   }
510 
511   /*
512    * Start up all services. If any of these threads gets an unhandled exception
513    * then they just die with a logged message.  This should be fine because
514    * in general, we do not expect the master to get such unhandled exceptions
515    *  as OOMEs; it should be lightly loaded. See what HRegionServer does if
516    *  need to install an unexpected exception handler.
517    */
518   private void startServiceThreads() {
519     try {
520       // Start the executor service pools
521       this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
522         conf.getInt("hbase.master.executor.openregion.threads", 5));
523       this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
524         conf.getInt("hbase.master.executor.closeregion.threads", 5));
525       this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
526         conf.getInt("hbase.master.executor.serverops.threads", 3));
527       this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
528         conf.getInt("hbase.master.executor.serverops.threads", 5));
529       // We depend on there being only one instance of this executor running
530       // at a time.  To do concurrency, would need fencing of enable/disable of
531       // tables.
532       this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
533 
534       // Start log cleaner thread
535       String n = Thread.currentThread().getName();
536       this.logCleaner =
537         new LogCleaner(conf.getInt("hbase.master.cleaner.interval", 60 * 1000),
538           this, conf, getMasterFileSystem().getFileSystem(),
539           getMasterFileSystem().getOldLogDir());
540       Threads.setDaemonThreadRunning(logCleaner, n + ".oldLogCleaner");
541 
542       // Put up info server.
543       int port = this.conf.getInt("hbase.master.info.port", 60010);
544       if (port >= 0) {
545         String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
546         this.infoServer = new InfoServer(MASTER, a, port, false);
547         this.infoServer.setAttribute(MASTER, this);
548         this.infoServer.start();
549       }
550       // Start allowing requests to happen.
551       this.rpcServer.openServer();
552       if (LOG.isDebugEnabled()) {
553         LOG.debug("Started service threads");
554       }
555     } catch (IOException e) {
556       if (e instanceof RemoteException) {
557         e = ((RemoteException)e).unwrapRemoteException();
558       }
559       // Something happened during startup. Shut things down.
560       abort("Failed startup", e);
561     }
562   }
563 
564   private void stopServiceThreads() {
565     if (LOG.isDebugEnabled()) {
566       LOG.debug("Stopping service threads");
567     }
568     if (this.rpcServer != null) this.rpcServer.stop();
569     // Clean up and close up shop
570     if (this.logCleaner!= null) this.logCleaner.interrupt();
571     if (this.infoServer != null) {
572       LOG.info("Stopping infoServer");
573       try {
574         this.infoServer.stop();
575       } catch (Exception ex) {
576         ex.printStackTrace();
577       }
578     }
579     if (this.executorService != null) this.executorService.shutdown();
580   }
581 
582   private static Thread getAndStartBalancerChore(final HMaster master) {
583     String name = master.getServerName() + "-BalancerChore";
584     int period = master.getConfiguration().getInt("hbase.balancer.period", 300000);
585     // Start up the load balancer chore
586     Chore chore = new Chore(name, period, master) {
587       @Override
588       protected void chore() {
589         master.balance();
590       }
591     };
592     return Threads.setDaemonThreadRunning(chore);
593   }
594 
595   private void stopChores() {
596     if (this.balancerChore != null) {
597       this.balancerChore.interrupt();
598     }
599     if (this.catalogJanitorChore != null) {
600       this.catalogJanitorChore.interrupt();
601     }
602   }
603 
604   @Override
605   public MapWritable regionServerStartup(final HServerInfo serverInfo,
606     final long serverCurrentTime)
607   throws IOException {
608     // Set the ip into the passed in serverInfo.  Its ip is more than likely
609     // not the ip that the master sees here.  See at end of this method where
610     // we pass it back to the regionserver by setting "hbase.regionserver.address"
611     // Everafter, the HSI combination 'server name' is what uniquely identifies
612     // the incoming RegionServer.
613     InetSocketAddress address = new InetSocketAddress(
614         HBaseServer.getRemoteIp().getHostName(),
615         serverInfo.getServerAddress().getPort());
616     serverInfo.setServerAddress(new HServerAddress(address));
617 
618     // Register with server manager
619     this.serverManager.regionServerStartup(serverInfo, serverCurrentTime);
620     // Send back some config info
621     MapWritable mw = createConfigurationSubset();
622      mw.put(new Text("hbase.regionserver.address"),
623          serverInfo.getServerAddress());
624     return mw;
625   }
626 
627   /**
628    * @return Subset of configuration to pass initializing regionservers: e.g.
629    * the filesystem to use and root directory to use.
630    */
631   protected MapWritable createConfigurationSubset() {
632     MapWritable mw = addConfig(new MapWritable(), HConstants.HBASE_DIR);
633     return addConfig(mw, "fs.default.name");
634   }
635 
636   private MapWritable addConfig(final MapWritable mw, final String key) {
637     mw.put(new Text(key), new Text(this.conf.get(key)));
638     return mw;
639   }
640 
641   @Override
642   public HMsg [] regionServerReport(HServerInfo serverInfo, HMsg msgs[],
643     HRegionInfo[] mostLoadedRegions)
644   throws IOException {
645     return adornRegionServerAnswer(serverInfo,
646       this.serverManager.regionServerReport(serverInfo, msgs, mostLoadedRegions));
647   }
648 
649   /**
650    * Override if you'd add messages to return to regionserver <code>hsi</code>
651    * or to send an exception.
652    * @param msgs Messages to add to
653    * @return Messages to return to
654    * @throws IOException exceptions that were injected for the region servers
655    */
656   protected HMsg [] adornRegionServerAnswer(final HServerInfo hsi,
657       final HMsg [] msgs) throws IOException {
658     return msgs;
659   }
660 
661   public boolean isMasterRunning() {
662     return !isStopped();
663   }
664 
665   @Override
666   public boolean balance() {
667     // If balance not true, don't run balancer.
668     if (!this.balanceSwitch) return false;
669     synchronized (this.balancer) {
670       // Only allow one balance run at at time.
671       if (this.assignmentManager.isRegionsInTransition()) {
672         LOG.debug("Not running balancer because " +
673           this.assignmentManager.getRegionsInTransition().size() +
674           " region(s) in transition: " +
675           org.apache.commons.lang.StringUtils.
676             abbreviate(this.assignmentManager.getRegionsInTransition().toString(), 256));
677         return false;
678       }
679       if (this.serverManager.areDeadServersInProgress()) {
680         LOG.debug("Not running balancer because processing dead regionserver(s): "  +
681           this.serverManager.getDeadServers());
682         return false;
683       }
684       Map<HServerInfo, List<HRegionInfo>> assignments =
685         this.assignmentManager.getAssignments();
686       // Returned Map from AM does not include mention of servers w/o assignments.
687       for (Map.Entry<String, HServerInfo> e:
688           this.serverManager.getOnlineServers().entrySet()) {
689         HServerInfo hsi = e.getValue();
690         if (!assignments.containsKey(hsi)) {
691           assignments.put(hsi, new ArrayList<HRegionInfo>());
692         }
693       }
694       List<RegionPlan> plans = this.balancer.balanceCluster(assignments);
695       if (plans != null && !plans.isEmpty()) {
696         for (RegionPlan plan: plans) {
697           LOG.info("balance " + plan);
698           this.assignmentManager.balance(plan);
699         }
700       }
701     }
702     return true;
703   }
704 
705   @Override
706   public boolean balanceSwitch(final boolean b) {
707     boolean oldValue = this.balanceSwitch;
708     this.balanceSwitch = b;
709     LOG.info("Balance=" + b);
710     return oldValue;
711   }
712 
713   /**
714    * Switch for the background {@link CatalogJanitor} thread.
715    * Used for testing.  The thread will continue to run.  It will just be a noop
716    * if disabled.
717    * @param b If false, the catalog janitor won't do anything.
718    */
719   public void setCatalogJanitorEnabled(final boolean b) {
720     ((CatalogJanitor)this.catalogJanitorChore).setEnabled(b);
721   }
722 
723   @Override
724   public void move(final byte[] encodedRegionName, final byte[] destServerName)
725   throws UnknownRegionException {
726     Pair<HRegionInfo, HServerInfo> p =
727       this.assignmentManager.getAssignment(encodedRegionName);
728     if (p == null)
729       throw new UnknownRegionException(Bytes.toString(encodedRegionName));
730     HRegionInfo hri = p.getFirst();
731     HServerInfo dest = null;
732     if (destServerName == null || destServerName.length == 0) {
733       LOG.info("Passed destination servername is null/empty so " +
734         "choosing a server at random");
735       this.assignmentManager.clearRegionPlan(hri);
736       // Unassign will reassign it elsewhere choosing random server.
737       this.assignmentManager.unassign(hri);
738     } else {
739       dest = this.serverManager.getServerInfo(new String(destServerName));
740       RegionPlan rp = new RegionPlan(p.getFirst(), p.getSecond(), dest);
741       this.assignmentManager.balance(rp);
742     }
743   }
744 
745   public void createTable(HTableDescriptor desc, byte [][] splitKeys)
746   throws IOException {
747     createTable(desc, splitKeys, false);
748   }
749 
750   public void createTable(HTableDescriptor desc, byte [][] splitKeys,
751       boolean sync)
752   throws IOException {
753     if (!isMasterRunning()) {
754       throw new MasterNotRunningException();
755     }
756     HRegionInfo [] newRegions = null;
757     if(splitKeys == null || splitKeys.length == 0) {
758       newRegions = new HRegionInfo [] { new HRegionInfo(desc, null, null) };
759     } else {
760       int numRegions = splitKeys.length + 1;
761       newRegions = new HRegionInfo[numRegions];
762       byte [] startKey = null;
763       byte [] endKey = null;
764       for(int i=0;i<numRegions;i++) {
765         endKey = (i == splitKeys.length) ? null : splitKeys[i];
766         newRegions[i] = new HRegionInfo(desc, startKey, endKey);
767         startKey = endKey;
768       }
769     }
770     int timeout = conf.getInt("hbase.client.catalog.timeout", 10000);
771     // Need META availability to create a table
772     try {
773       if(catalogTracker.waitForMeta(timeout) == null) {
774         throw new NotAllMetaRegionsOnlineException();
775       }
776     } catch (InterruptedException e) {
777       LOG.warn("Interrupted waiting for meta availability", e);
778       throw new IOException(e);
779     }
780     createTable(newRegions, sync);
781   }
782 
783   private synchronized void createTable(final HRegionInfo [] newRegions,
784       boolean sync)
785   throws IOException {
786     String tableName = newRegions[0].getTableDesc().getNameAsString();
787     if(MetaReader.tableExists(catalogTracker, tableName)) {
788       throw new TableExistsException(tableName);
789     }
790     for(HRegionInfo newRegion : newRegions) {
791 
792       // 1. Set table enabling flag up in zk.
793       try {
794         assignmentManager.getZKTable().setEnabledTable(tableName);
795       } catch (KeeperException e) {
796         throw new IOException("Unable to ensure that the table will be" +
797             " enabled because of a ZooKeeper issue", e);
798       }
799 
800       // 2. Create HRegion
801       HRegion region = HRegion.createHRegion(newRegion,
802           fileSystemManager.getRootDir(), conf);
803 
804       // 3. Insert into META
805       MetaEditor.addRegionToMeta(catalogTracker, region.getRegionInfo());
806 
807       // 4. Close the new region to flush to disk.  Close log file too.
808       region.close();
809       region.getLog().closeAndDelete();
810     }
811 
812     // 5. Trigger immediate assignment of the regions in round-robin fashion
813     List<HServerInfo> servers = serverManager.getOnlineServersList();
814     try {
815       this.assignmentManager.assignUserRegions(Arrays.asList(newRegions), servers);
816     } catch (InterruptedException ie) {
817       LOG.error("Caught " + ie + " during round-robin assignment");
818       throw new IOException(ie);
819     }
820 
821     // 5. If sync, wait for assignment of regions
822     if(sync) {
823       LOG.debug("Waiting for " + newRegions.length + " region(s) to be " +
824           "assigned before returning");
825       for(HRegionInfo regionInfo : newRegions) {
826         try {
827           assignmentManager.waitForAssignment(regionInfo);
828         } catch (InterruptedException e) {
829           LOG.info("Interrupted waiting for region to be assigned during " +
830               "create table call");
831           return;
832         }
833       }
834     }
835   }
836 
837   private static boolean isCatalogTable(final byte [] tableName) {
838     return Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) ||
839            Bytes.equals(tableName, HConstants.META_TABLE_NAME);
840   }
841 
842   public void deleteTable(final byte [] tableName) throws IOException {
843     this.executorService.submit(new DeleteTableHandler(tableName, this, this));
844   }
845 
846   public void addColumn(byte [] tableName, HColumnDescriptor column)
847   throws IOException {
848     new TableAddFamilyHandler(tableName, column, this, this).process();
849   }
850 
851   public void modifyColumn(byte [] tableName, HColumnDescriptor descriptor)
852   throws IOException {
853     new TableModifyFamilyHandler(tableName, descriptor, this, this).process();
854   }
855 
856   public void deleteColumn(final byte [] tableName, final byte [] c)
857   throws IOException {
858     new TableDeleteFamilyHandler(tableName, c, this, this).process();
859   }
860 
861   public void enableTable(final byte [] tableName) throws IOException {
862     this.executorService.submit(new EnableTableHandler(this, tableName,
863       catalogTracker, assignmentManager));
864   }
865 
866   public void disableTable(final byte [] tableName) throws IOException {
867     this.executorService.submit(new DisableTableHandler(this, tableName,
868       catalogTracker, assignmentManager));
869   }
870 
871   /**
872    * Return the region and current deployment for the region containing
873    * the given row. If the region cannot be found, returns null. If it
874    * is found, but not currently deployed, the second element of the pair
875    * may be null.
876    */
877   Pair<HRegionInfo,HServerAddress> getTableRegionForRow(
878       final byte [] tableName, final byte [] rowKey)
879   throws IOException {
880     final AtomicReference<Pair<HRegionInfo, HServerAddress>> result =
881       new AtomicReference<Pair<HRegionInfo, HServerAddress>>(null);
882 
883     MetaScannerVisitor visitor =
884       new MetaScannerVisitor() {
885         @Override
886         public boolean processRow(Result data) throws IOException {
887           if (data == null || data.size() <= 0) {
888             return true;
889           }
890           Pair<HRegionInfo, HServerAddress> pair =
891             MetaReader.metaRowToRegionPair(data);
892           if (pair == null) {
893             return false;
894           }
895           if (!Bytes.equals(pair.getFirst().getTableDesc().getName(),
896                 tableName)) {
897             return false;
898           }
899           result.set(pair);
900           return true;
901         }
902     };
903 
904     MetaScanner.metaScan(conf, visitor, tableName, rowKey, 1);
905     return result.get();
906   }
907 
908   @Override
909   public void modifyTable(final byte[] tableName, HTableDescriptor htd)
910   throws IOException {
911     this.executorService.submit(new ModifyTableHandler(tableName, htd, this, this));
912   }
913 
914   @Override
915   public void checkTableModifiable(final byte [] tableName)
916   throws IOException {
917     String tableNameStr = Bytes.toString(tableName);
918     if (isCatalogTable(tableName)) {
919       throw new IOException("Can't modify catalog tables");
920     }
921     if (!MetaReader.tableExists(getCatalogTracker(), tableNameStr)) {
922       throw new TableNotFoundException(tableNameStr);
923     }
924     if (!getAssignmentManager().getZKTable().
925         isDisabledTable(Bytes.toString(tableName))) {
926       throw new TableNotDisabledException(tableName);
927     }
928   }
929 
930   public void clearFromTransition(HRegionInfo hri) {
931     if (this.assignmentManager.isRegionInTransition(hri) != null) {
932       this.assignmentManager.clearRegionFromTransition(hri);
933     }
934   }
935   /**
936    * @return cluster status
937    */
938   public ClusterStatus getClusterStatus() {
939     ClusterStatus status = new ClusterStatus();
940     status.setHBaseVersion(VersionInfo.getVersion());
941     status.setServerInfo(serverManager.getOnlineServers().values());
942     status.setDeadServers(serverManager.getDeadServers());
943     status.setRegionsInTransition(assignmentManager.getRegionsInTransition());
944     return status;
945   }
946 
947   @Override
948   public void abort(final String msg, final Throwable t) {
949     if (t != null) LOG.fatal(msg, t);
950     else LOG.fatal(msg);
951     this.abort = true;
952     stop("Aborting");
953   }
954 
955   @Override
956   public ZooKeeperWatcher getZooKeeper() {
957     return zooKeeper;
958   }
959 
960   @Override
961   public String getServerName() {
962     return address.toString();
963   }
964 
965   @Override
966   public CatalogTracker getCatalogTracker() {
967     return catalogTracker;
968   }
969 
970   @Override
971   public AssignmentManager getAssignmentManager() {
972     return this.assignmentManager;
973   }
974 
975   @Override
976   public void shutdown() {
977     this.serverManager.shutdownCluster();
978     try {
979       this.clusterStatusTracker.setClusterDown();
980     } catch (KeeperException e) {
981       LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
982     }
983   }
984 
985   @Override
986   public void stopMaster() {
987     stop("Stopped by " + Thread.currentThread().getName());
988   }
989 
990   @Override
991   public void stop(final String why) {
992     LOG.info(why);
993     this.stopped = true;
994     // If we are a backup master, we need to interrupt wait
995     synchronized (this.activeMasterManager.clusterHasActiveMaster) {
996       this.activeMasterManager.clusterHasActiveMaster.notifyAll();
997     }
998   }
999 
1000   @Override
1001   public boolean isStopped() {
1002     return this.stopped;
1003   }
1004 
1005   /**
1006    * Report whether this master is currently the active master or not.
1007    * If not active master, we are parked on ZK waiting to become active.
1008    *
1009    * This method is used for testing.
1010    *
1011    * @return true if active master, false if not.
1012    */
1013   public boolean isActiveMaster() {
1014     return isActiveMaster;
1015   }
1016 
1017   /**
1018    * Report whether this master has completed with its initialization and is
1019    * ready.  If ready, the master is also the active master.  A standby master
1020    * is never ready.
1021    *
1022    * This method is used for testing.
1023    *
1024    * @return true if master is ready to go, false if not.
1025    */
1026   public boolean isInitialized() {
1027     return initialized;
1028   }
1029 
1030   @Override
1031   public void assign(final byte [] regionName, final boolean force)
1032   throws IOException {
1033     Pair<HRegionInfo, HServerAddress> pair =
1034       MetaReader.getRegion(this.catalogTracker, regionName);
1035     if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
1036     assignRegion(pair.getFirst());
1037   }
1038 
1039   public void assignRegion(HRegionInfo hri) {
1040     assignmentManager.assign(hri, true);
1041   }
1042 
1043   @Override
1044   public void unassign(final byte [] regionName, final boolean force)
1045   throws IOException {
1046     Pair<HRegionInfo, HServerAddress> pair =
1047       MetaReader.getRegion(this.catalogTracker, regionName);
1048     if (pair == null) throw new UnknownRegionException(Bytes.toString(regionName));
1049     HRegionInfo hri = pair.getFirst();
1050     if (force) this.assignmentManager.clearRegionFromTransition(hri);
1051     this.assignmentManager.unassign(hri, force);
1052   }
1053 
1054   /**
1055    * Utility for constructing an instance of the passed HMaster class.
1056    * @param masterClass
1057    * @param conf
1058    * @return HMaster instance.
1059    */
1060   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
1061       final Configuration conf)  {
1062     try {
1063       Constructor<? extends HMaster> c =
1064         masterClass.getConstructor(Configuration.class);
1065       return c.newInstance(conf);
1066     } catch (InvocationTargetException ite) {
1067       Throwable target = ite.getTargetException() != null?
1068         ite.getTargetException(): ite;
1069       if (target.getCause() != null) target = target.getCause();
1070       throw new RuntimeException("Failed construction of Master: " +
1071         masterClass.toString(), target);
1072     } catch (Exception e) {
1073       throw new RuntimeException("Failed construction of Master: " +
1074         masterClass.toString() + ((e.getCause() != null)?
1075           e.getCause().getMessage(): ""), e);
1076     }
1077   }
1078 
1079 
1080   /**
1081    * @see org.apache.hadoop.hbase.master.HMasterCommandLine
1082    */
1083   public static void main(String [] args) throws Exception {
1084     new HMasterCommandLine(HMaster.class).doMain(args);
1085   }
1086 }