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