View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.master;
20  
21  import com.google.common.annotations.VisibleForTesting;
22  import com.google.common.cache.Cache;
23  import com.google.common.cache.CacheBuilder;
24  import com.google.common.collect.Lists;
25  import com.google.common.collect.Maps;
26  import com.google.protobuf.Descriptors;
27  import com.google.protobuf.Service;
28  
29  import java.io.IOException;
30  import java.io.InterruptedIOException;
31  import java.lang.reflect.Constructor;
32  import java.lang.reflect.InvocationTargetException;
33  import java.net.InetAddress;
34  import java.net.InetSocketAddress;
35  import java.net.UnknownHostException;
36  import java.util.ArrayList;
37  import java.util.Collection;
38  import java.util.Collections;
39  import java.util.Comparator;
40  import java.util.HashSet;
41  import java.util.Iterator;
42  import java.util.List;
43  import java.util.Map;
44  import java.util.Map.Entry;
45  import java.util.Set;
46  import java.util.concurrent.CountDownLatch;
47  import java.util.concurrent.TimeUnit;
48  import java.util.concurrent.atomic.AtomicInteger;
49  import java.util.concurrent.atomic.AtomicReference;
50  import java.util.regex.Pattern;
51  
52  import javax.servlet.ServletException;
53  import javax.servlet.http.HttpServlet;
54  import javax.servlet.http.HttpServletRequest;
55  import javax.servlet.http.HttpServletResponse;
56  
57  import org.apache.commons.logging.Log;
58  import org.apache.commons.logging.LogFactory;
59  import org.apache.hadoop.conf.Configuration;
60  import org.apache.hadoop.fs.FileSystem;
61  import org.apache.hadoop.fs.Path;
62  import org.apache.hadoop.hbase.ClusterStatus;
63  import org.apache.hadoop.hbase.CoordinatedStateException;
64  import org.apache.hadoop.hbase.CoordinatedStateManager;
65  import org.apache.hadoop.hbase.DoNotRetryIOException;
66  import org.apache.hadoop.hbase.HBaseIOException;
67  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
68  import org.apache.hadoop.hbase.HColumnDescriptor;
69  import org.apache.hadoop.hbase.HConstants;
70  import org.apache.hadoop.hbase.HRegionInfo;
71  import org.apache.hadoop.hbase.HTableDescriptor;
72  import org.apache.hadoop.hbase.MasterNotRunningException;
73  import org.apache.hadoop.hbase.MetaMigrationConvertingToPB;
74  import org.apache.hadoop.hbase.MetaTableAccessor;
75  import org.apache.hadoop.hbase.NamespaceDescriptor;
76  import org.apache.hadoop.hbase.NamespaceNotFoundException;
77  import org.apache.hadoop.hbase.PleaseHoldException;
78  import org.apache.hadoop.hbase.ProcedureInfo;
79  import org.apache.hadoop.hbase.Server;
80  import org.apache.hadoop.hbase.ServerLoad;
81  import org.apache.hadoop.hbase.ServerName;
82  import org.apache.hadoop.hbase.TableDescriptors;
83  import org.apache.hadoop.hbase.TableName;
84  import org.apache.hadoop.hbase.TableNotDisabledException;
85  import org.apache.hadoop.hbase.TableNotFoundException;
86  import org.apache.hadoop.hbase.TableStateManager;
87  import org.apache.hadoop.hbase.UnknownRegionException;
88  import org.apache.hadoop.hbase.backup.BackupType;
89  import org.apache.hadoop.hbase.backup.HBackupFileSystem;
90  import org.apache.hadoop.hbase.backup.impl.BackupManager;
91  import org.apache.hadoop.hbase.backup.impl.BackupRestoreConstants;
92  import org.apache.hadoop.hbase.backup.impl.BackupSystemTable;
93  import org.apache.hadoop.hbase.backup.master.FullTableBackupProcedure;
94  import org.apache.hadoop.hbase.backup.master.IncrementalTableBackupProcedure;
95  import org.apache.hadoop.hbase.classification.InterfaceAudience;
96  import org.apache.hadoop.hbase.client.Admin;
97  import org.apache.hadoop.hbase.client.MetaScanner;
98  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
99  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
100 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
101 import org.apache.hadoop.hbase.client.Result;
102 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
103 import org.apache.hadoop.hbase.exceptions.DeserializationException;
104 import org.apache.hadoop.hbase.executor.ExecutorType;
105 import org.apache.hadoop.hbase.ipc.RpcServer;
106 import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
107 import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
108 import org.apache.hadoop.hbase.master.RegionState.State;
109 import org.apache.hadoop.hbase.master.balancer.BalancerChore;
110 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
111 import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
112 import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
113 import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
114 import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
115 import org.apache.hadoop.hbase.master.handler.CreateTableHandler;
116 import org.apache.hadoop.hbase.master.handler.DeleteTableHandler;
117 import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
118 import org.apache.hadoop.hbase.master.handler.DispatchMergingRegionHandler;
119 import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
120 import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
121 import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
122 import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
123 import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
124 import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
125 import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan;
126 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizer;
127 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerChore;
128 import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
129 import org.apache.hadoop.hbase.master.procedure.AddColumnFamilyProcedure;
130 import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
131 import org.apache.hadoop.hbase.master.procedure.DeleteColumnFamilyProcedure;
132 import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
133 import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
134 import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
135 import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
136 import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
137 import org.apache.hadoop.hbase.master.procedure.ModifyColumnFamilyProcedure;
138 import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
139 import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
140 import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
141 import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
142 import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
143 import org.apache.hadoop.hbase.mob.MobConstants;
144 import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
145 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
146 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
147 import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
148 import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
149 import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
150 import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
151 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
152 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionServerInfo;
153 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
154 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
155 import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
156 import org.apache.hadoop.hbase.quotas.RegionStateListener;
157 import org.apache.hadoop.hbase.regionserver.DefaultStoreEngine;
158 import org.apache.hadoop.hbase.regionserver.HRegionServer;
159 import org.apache.hadoop.hbase.regionserver.HStore;
160 import org.apache.hadoop.hbase.regionserver.RSRpcServices;
161 import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
162 import org.apache.hadoop.hbase.regionserver.RegionSplitPolicy;
163 import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
164 import org.apache.hadoop.hbase.regionserver.compactions.FIFOCompactionPolicy;
165 import org.apache.hadoop.hbase.replication.regionserver.Replication;
166 import org.apache.hadoop.hbase.security.UserProvider;
167 import org.apache.hadoop.hbase.util.Addressing;
168 import org.apache.hadoop.hbase.util.Bytes;
169 import org.apache.hadoop.hbase.util.CompressionTest;
170 import org.apache.hadoop.hbase.util.ConfigUtil;
171 import org.apache.hadoop.hbase.util.EncryptionTest;
172 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
173 import org.apache.hadoop.hbase.util.FSUtils;
174 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
175 import org.apache.hadoop.hbase.util.HasThread;
176 import org.apache.hadoop.hbase.util.IdLock;
177 import org.apache.hadoop.hbase.util.ModifyRegionUtils;
178 import org.apache.hadoop.hbase.util.Pair;
179 import org.apache.hadoop.hbase.util.Threads;
180 import org.apache.hadoop.hbase.util.VersionInfo;
181 import org.apache.hadoop.hbase.zookeeper.DrainingServerTracker;
182 import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
183 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
184 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
185 import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
186 import org.apache.hadoop.hbase.zookeeper.RegionServerTracker;
187 import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTracker;
188 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
189 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
190 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
191 import org.apache.zookeeper.KeeperException;
192 import org.mortbay.jetty.Connector;
193 import org.mortbay.jetty.nio.SelectChannelConnector;
194 import org.mortbay.jetty.servlet.Context;
195 
196 /**
197  * HMaster is the "master server" for HBase. An HBase cluster has one active
198  * master.  If many masters are started, all compete.  Whichever wins goes on to
199  * run the cluster.  All others park themselves in their constructor until
200  * master or cluster shutdown or until the active master loses its lease in
201  * zookeeper.  Thereafter, all running master jostle to take over master role.
202  *
203  * <p>The Master can be asked shutdown the cluster. See {@link #shutdown()}.  In
204  * this case it will tell all regionservers to go down and then wait on them
205  * all reporting in that they are down.  This master will then shut itself down.
206  *
207  * <p>You can also shutdown just this master.  Call {@link #stopMaster()}.
208  *
209  * @see org.apache.zookeeper.Watcher
210  */
211 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
212 @SuppressWarnings("deprecation")
213 public class HMaster extends HRegionServer implements MasterServices, Server {
214   private static final Log LOG = LogFactory.getLog(HMaster.class.getName());
215 
216   /**
217    * Protection against zombie master. Started once Master accepts active responsibility and
218    * starts taking over responsibilities. Allows a finite time window before giving up ownership.
219    */
220   private static class InitializationMonitor extends HasThread {
221     /** The amount of time in milliseconds to sleep before checking initialization status. */
222     public static final String TIMEOUT_KEY = "hbase.master.initializationmonitor.timeout";
223     public static final long TIMEOUT_DEFAULT = TimeUnit.MILLISECONDS.convert(15, TimeUnit.MINUTES);
224 
225     /**
226      * When timeout expired and initialization has not complete, call {@link System#exit(int)} when
227      * true, do nothing otherwise.
228      */
229     public static final String HALT_KEY = "hbase.master.initializationmonitor.haltontimeout";
230     public static final boolean HALT_DEFAULT = false;
231 
232     private final HMaster master;
233     private final long timeout;
234     private final boolean haltOnTimeout;
235 
236     /** Creates a Thread that monitors the {@link #isInitialized()} state. */
237     InitializationMonitor(HMaster master) {
238       super("MasterInitializationMonitor");
239       this.master = master;
240       this.timeout = master.getConfiguration().getLong(TIMEOUT_KEY, TIMEOUT_DEFAULT);
241       this.haltOnTimeout = master.getConfiguration().getBoolean(HALT_KEY, HALT_DEFAULT);
242       this.setDaemon(true);
243     }
244 
245     @Override
246     public void run() {
247       try {
248         while (!master.isStopped() && master.isActiveMaster()) {
249           Thread.sleep(timeout);
250           if (master.isInitialized()) {
251             LOG.debug("Initialization completed within allotted tolerance. Monitor exiting.");
252           } else {
253             LOG.error("Master failed to complete initialization after " + timeout + "ms. Please"
254                 + " consider submitting a bug report including a thread dump of this process.");
255             if (haltOnTimeout) {
256               LOG.error("Zombie Master exiting. Thread dump to stdout");
257               Threads.printThreadInfo(System.out, "Zombie HMaster");
258               System.exit(-1);
259             }
260           }
261         }
262       } catch (InterruptedException ie) {
263         LOG.trace("InitMonitor thread interrupted. Existing.");
264       }
265     }
266   }
267 
268   // MASTER is name of the webapp and the attribute name used stuffing this
269   //instance into web context.
270   public static final String MASTER = "master";
271 
272   // Manager and zk listener for master election
273   private final ActiveMasterManager activeMasterManager;
274   // Region server tracker
275   RegionServerTracker regionServerTracker;
276   // Draining region server tracker
277   private DrainingServerTracker drainingServerTracker;
278   // Tracker for load balancer state
279   LoadBalancerTracker loadBalancerTracker;
280 
281   // Tracker for split and merge state
282   private SplitOrMergeTracker splitOrMergeTracker;
283 
284   // Tracker for region normalizer state
285   private RegionNormalizerTracker regionNormalizerTracker;
286 
287   /** Namespace stuff */
288   private TableNamespaceManager tableNamespaceManager;
289 
290   // Metrics for the HMaster
291   final MetricsMaster metricsMaster;
292   // file system manager for the master FS operations
293   private MasterFileSystem fileSystemManager;
294 
295   // server manager to deal with region server info
296   volatile ServerManager serverManager;
297 
298   // manager of assignment nodes in zookeeper
299   AssignmentManager assignmentManager;
300 
301   // buffer for "fatal error" notices from region servers
302   // in the cluster. This is only used for assisting
303   // operations/debugging.
304   MemoryBoundedLogMessageBuffer rsFatals;
305 
306   // flag set after we become the active master (used for testing)
307   private volatile boolean isActiveMaster = false;
308 
309   // flag set after we complete initialization once active,
310   // it is not private since it's used in unit tests
311   volatile boolean initialized = false;
312 
313   // flag set after master services are started,
314   // initialization may have not completed yet.
315   volatile boolean serviceStarted = false;
316 
317   // flag set after we complete assignMeta.
318   private volatile boolean serverShutdownHandlerEnabled = false;
319 
320   LoadBalancer balancer;
321   private RegionNormalizer normalizer;
322   private BalancerChore balancerChore;
323   private RegionNormalizerChore normalizerChore;
324   private ClusterStatusChore clusterStatusChore;
325   private ClusterStatusPublisher clusterStatusPublisherChore = null;
326 
327   CatalogJanitor catalogJanitorChore;
328   private LogCleaner logCleaner;
329   private HFileCleaner hfileCleaner;
330   private ExpiredMobFileCleanerChore expiredMobFileCleanerChore;
331   private MobCompactionChore mobCompactChore;
332   private MasterMobCompactionThread mobCompactThread;
333   // used to synchronize the mobCompactionStates
334   private final IdLock mobCompactionLock = new IdLock();
335   // save the information of mob compactions in tables.
336   // the key is table name, the value is the number of compactions in that table.
337   private Map<TableName, AtomicInteger> mobCompactionStates = Maps.newConcurrentMap();
338 
339   MasterCoprocessorHost cpHost;
340 
341   private final boolean preLoadTableDescriptors;
342 
343   // Time stamps for when a hmaster became active
344   private long masterActiveTime;
345 
346   //should we check the compression codec type at master side, default true, HBASE-6370
347   private final boolean masterCheckCompression;
348 
349   //should we check encryption settings at master side, default true
350   private final boolean masterCheckEncryption;
351 
352   // This is for fallback to use the code from 1.0 release.
353   private enum ProcedureConf {
354     PROCEDURE_ENABLED, // default
355     HANDLER_USED, // handler code executed in DDL, procedure executor still start
356     PROCEDURE_FULLY_DISABLED, // procedure fully disabled
357   }
358   private final ProcedureConf procedureConf;
359 
360   Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
361 
362   // monitor for snapshot of hbase tables
363   SnapshotManager snapshotManager;
364   // monitor for distributed procedures
365   private MasterProcedureManagerHost mpmHost;
366 
367   // it is assigned after 'initialized' guard set to true, so should be volatile
368   private volatile MasterQuotaManager quotaManager;
369 
370   private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
371   private WALProcedureStore procedureStore;
372 
373   /** flag used in test cases in order to simulate RS failures during master initialization */
374   private volatile boolean initializationBeforeMetaAssignment = false;
375 
376   /** jetty server for master to redirect requests to regionserver infoServer */
377   private org.mortbay.jetty.Server masterJettyServer;
378 
379   public static class RedirectServlet extends HttpServlet {
380     private static final long serialVersionUID = 2894774810058302472L;
381     private static int regionServerInfoPort;
382 
383     @Override
384     public void doGet(HttpServletRequest request,
385         HttpServletResponse response) throws ServletException, IOException {
386       String redirectUrl = request.getScheme() + "://"
387         + request.getServerName() + ":" + regionServerInfoPort
388         + request.getRequestURI();
389       response.sendRedirect(redirectUrl);
390     }
391   }
392 
393   /**
394    * Initializes the HMaster. The steps are as follows:
395    * <p>
396    * <ol>
397    * <li>Initialize the local HRegionServer
398    * <li>Start the ActiveMasterManager.
399    * </ol>
400    * <p>
401    * Remaining steps of initialization occur in
402    * #finishActiveMasterInitialization(MonitoredTask) after
403    * the master becomes the active one.
404    *
405    * @throws InterruptedException
406    * @throws KeeperException
407    * @throws IOException
408    */
409   public HMaster(final Configuration conf, CoordinatedStateManager csm)
410       throws IOException, KeeperException, InterruptedException {
411     super(conf, csm);
412     this.rsFatals = new MemoryBoundedLogMessageBuffer(
413       conf.getLong("hbase.master.buffer.for.rs.fatals", 1*1024*1024));
414 
415     LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) +
416         ", hbase.cluster.distributed=" + this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));
417 
418     // Disable usage of meta replicas in the master
419     this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
420 
421     Replication.decorateMasterConfiguration(this.conf);
422     BackupManager.decorateMasterConfiguration(this.conf);
423 
424     // Hack! Maps DFSClient => Master for logs.  HDFS made this
425     // config param for task trackers, but we can piggyback off of it.
426     if (this.conf.get("mapreduce.task.attempt.id") == null) {
427       this.conf.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName.toString());
428     }
429 
430     // should we check the compression codec type at master side, default true, HBASE-6370
431     this.masterCheckCompression = conf.getBoolean("hbase.master.check.compression", true);
432 
433     // should we check encryption settings at master side, default true
434     this.masterCheckEncryption = conf.getBoolean("hbase.master.check.encryption", true);
435 
436     this.metricsMaster = new MetricsMaster(new MetricsMasterWrapperImpl(this));
437 
438     // Check configuration to see whether procedure is disabled (not execute at all),
439     // unused (not used to execute DDL, but executor starts to complete unfinished operations
440     // in procedure store, or enabled (default behavior).
441     String procedureConfString = conf.get("hbase.master.procedure.tableddl", "enabled");
442     if (procedureConfString.equalsIgnoreCase("disabled")) {
443       LOG.info("Master will use handler for new table DDL"
444         + " and all unfinished table DDLs in procedure store will be disgarded.");
445       this.procedureConf = ProcedureConf.PROCEDURE_FULLY_DISABLED;
446     } else if (procedureConfString.equalsIgnoreCase("unused")) {
447       LOG.info("Master will use handler for new table DDL"
448         + " and all unfinished table DDLs in procedure store will continue to execute.");
449       this.procedureConf = ProcedureConf.HANDLER_USED;
450     } else {
451       this.procedureConf = ProcedureConf.PROCEDURE_ENABLED;
452     }
453     // preload table descriptor at startup
454     this.preLoadTableDescriptors = conf.getBoolean("hbase.master.preload.tabledescriptors", true);
455 
456     // Do we publish the status?
457 
458     boolean shouldPublish = conf.getBoolean(HConstants.STATUS_PUBLISHED,
459         HConstants.STATUS_PUBLISHED_DEFAULT);
460     Class<? extends ClusterStatusPublisher.Publisher> publisherClass =
461         conf.getClass(ClusterStatusPublisher.STATUS_PUBLISHER_CLASS,
462             ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS,
463             ClusterStatusPublisher.Publisher.class);
464 
465     if (shouldPublish) {
466       if (publisherClass == null) {
467         LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but " +
468             ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS +
469             " is not set - not publishing status");
470       } else {
471         clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);
472         getChoreService().scheduleChore(clusterStatusPublisherChore);
473       }
474     }
475 
476     // Some unit tests don't need a cluster, so no zookeeper at all
477     if (!conf.getBoolean("hbase.testing.nocluster", false)) {
478       setInitLatch(new CountDownLatch(1));
479       activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this);
480       int infoPort = putUpJettyServer();
481       startActiveMasterManager(infoPort);
482     } else {
483       activeMasterManager = null;
484     }
485   }
486 
487   // return the actual infoPort, -1 means disable info server.
488   private int putUpJettyServer() throws IOException {
489     if (!conf.getBoolean("hbase.master.infoserver.redirect", true)) {
490       return -1;
491     }
492     int infoPort = conf.getInt("hbase.master.info.port.orig",
493       HConstants.DEFAULT_MASTER_INFOPORT);
494     // -1 is for disabling info server, so no redirecting
495     if (infoPort < 0 || infoServer == null) {
496       return -1;
497     }
498     String addr = conf.get("hbase.master.info.bindAddress", "0.0.0.0");
499     if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {
500       String msg =
501           "Failed to start redirecting jetty server. Address " + addr
502               + " does not belong to this host. Correct configuration parameter: "
503               + "hbase.master.info.bindAddress";
504       LOG.error(msg);
505       throw new IOException(msg);
506     }
507 
508     RedirectServlet.regionServerInfoPort = infoServer.getPort();
509     if(RedirectServlet.regionServerInfoPort == infoPort) {
510       return infoPort;
511     }
512     masterJettyServer = new org.mortbay.jetty.Server();
513     Connector connector = new SelectChannelConnector();
514     connector.setHost(addr);
515     connector.setPort(infoPort);
516     masterJettyServer.addConnector(connector);
517     masterJettyServer.setStopAtShutdown(true);
518     Context context = new Context(masterJettyServer, "/", Context.NO_SESSIONS);
519     context.addServlet(RedirectServlet.class, "/*");
520     try {
521       masterJettyServer.start();
522     } catch (Exception e) {
523       throw new IOException("Failed to start redirecting jetty server", e);
524     }
525     return connector.getLocalPort();
526   }
527 
528   /**
529    * For compatibility, if failed with regionserver credentials, try the master one
530    */
531   @Override
532   protected void login(UserProvider user, String host) throws IOException {
533     try {
534       super.login(user, host);
535     } catch (IOException ie) {
536       user.login("hbase.master.keytab.file",
537         "hbase.master.kerberos.principal", host);
538     }
539   }
540 
541   /**
542    * If configured to put regions on active master,
543    * wait till a backup master becomes active.
544    * Otherwise, loop till the server is stopped or aborted.
545    */
546   @Override
547   protected void waitForMasterActive(){
548     boolean tablesOnMaster = BaseLoadBalancer.tablesOnMaster(conf);
549     while (!(tablesOnMaster && isActiveMaster)
550         && !isStopped() && !isAborted()) {
551       sleeper.sleep();
552     }
553   }
554 
555   @VisibleForTesting
556   public MasterRpcServices getMasterRpcServices() {
557     return (MasterRpcServices)rpcServices;
558   }
559 
560   public boolean balanceSwitch(final boolean b) throws IOException {
561     return getMasterRpcServices().switchBalancer(b, BalanceSwitchMode.ASYNC);
562   }
563 
564   @Override
565   protected String getProcessName() {
566     return MASTER;
567   }
568 
569   @Override
570   protected boolean canCreateBaseZNode() {
571     return true;
572   }
573 
574   @Override
575   protected boolean canUpdateTableDescriptor() {
576     return true;
577   }
578 
579   @Override
580   protected RSRpcServices createRpcServices() throws IOException {
581     return new MasterRpcServices(this);
582   }
583 
584   @Override
585   protected void configureInfoServer() {
586     infoServer.addServlet("master-status", "/master-status", MasterStatusServlet.class);
587     infoServer.setAttribute(MASTER, this);
588     if (BaseLoadBalancer.tablesOnMaster(conf)) {
589       super.configureInfoServer();
590     }
591   }
592 
593   @Override
594   protected Class<? extends HttpServlet> getDumpServlet() {
595     return MasterDumpServlet.class;
596   }
597 
598   /**
599    * Emit the HMaster metrics, such as region in transition metrics.
600    * Surrounding in a try block just to be sure metrics doesn't abort HMaster.
601    */
602   @Override
603   protected void doMetrics() {
604     try {
605       if (assignmentManager != null) {
606         assignmentManager.updateRegionsInTransitionMetrics();
607       }
608     } catch (Throwable e) {
609       LOG.error("Couldn't update metrics: " + e.getMessage());
610     }
611   }
612 
613   MetricsMaster getMasterMetrics() {
614     return metricsMaster;
615   }
616 
617   /**
618    * Initialize all ZK based system trackers.
619    * @throws IOException
620    * @throws InterruptedException
621    * @throws KeeperException
622    * @throws CoordinatedStateException
623    */
624   void initializeZKBasedSystemTrackers() throws IOException,
625       InterruptedException, KeeperException, CoordinatedStateException {
626     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
627     this.normalizer = RegionNormalizerFactory.getRegionNormalizer(conf);
628     this.normalizer.setMasterServices(this);
629     this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
630     this.loadBalancerTracker.start();
631 
632     this.regionNormalizerTracker = new RegionNormalizerTracker(zooKeeper, this);
633     this.regionNormalizerTracker.start();
634 
635     this.splitOrMergeTracker = new SplitOrMergeTracker(zooKeeper, conf, this);
636     this.splitOrMergeTracker.start();
637 
638     this.assignmentManager = new AssignmentManager(this, serverManager,
639       this.balancer, this.service, this.metricsMaster,
640       this.tableLockManager);
641     zooKeeper.registerListenerFirst(assignmentManager);
642 
643     this.regionServerTracker = new RegionServerTracker(zooKeeper, this,
644         this.serverManager);
645     this.regionServerTracker.start();
646 
647     this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this,
648       this.serverManager);
649     this.drainingServerTracker.start();
650 
651     // Set the cluster as up.  If new RSs, they'll be waiting on this before
652     // going ahead with their startup.
653     boolean wasUp = this.clusterStatusTracker.isClusterUp();
654     if (!wasUp) this.clusterStatusTracker.setClusterUp();
655 
656     LOG.info("Server active/primary master=" + this.serverName +
657         ", sessionid=0x" +
658         Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId()) +
659         ", setting cluster-up flag (Was=" + wasUp + ")");
660 
661     // create/initialize the snapshot manager and other procedure managers
662     this.snapshotManager = new SnapshotManager();
663     this.mpmHost = new MasterProcedureManagerHost();
664     this.mpmHost.register(this.snapshotManager);
665     this.mpmHost.register(new MasterFlushTableProcedureManager());
666     this.mpmHost.loadProcedures(conf);
667     this.mpmHost.initialize(this, this.metricsMaster);
668   }
669 
670   /**
671    * Finish initialization of HMaster after becoming the primary master.
672    *
673    * <ol>
674    * <li>Initialize master components - file system manager, server manager,
675    *     assignment manager, region server tracker, etc</li>
676    * <li>Start necessary service threads - balancer, catalog janior,
677    *     executor services, etc</li>
678    * <li>Set cluster as UP in ZooKeeper</li>
679    * <li>Wait for RegionServers to check-in</li>
680    * <li>Split logs and perform data recovery, if necessary</li>
681    * <li>Ensure assignment of meta/namespace regions<li>
682    * <li>Handle either fresh cluster start or master failover</li>
683    * </ol>
684    *
685    * @throws IOException
686    * @throws InterruptedException
687    * @throws KeeperException
688    * @throws CoordinatedStateException
689    */
690   private void finishActiveMasterInitialization(MonitoredTask status)
691       throws IOException, InterruptedException, KeeperException, CoordinatedStateException {
692 
693     isActiveMaster = true;
694     Thread zombieDetector = new Thread(new InitializationMonitor(this));
695     zombieDetector.start();
696 
697     /*
698      * We are active master now... go initialize components we need to run.
699      * Note, there may be dross in zk from previous runs; it'll get addressed
700      * below after we determine if cluster startup or failover.
701      */
702 
703     status.setStatus("Initializing Master file system");
704 
705     this.masterActiveTime = System.currentTimeMillis();
706     // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
707     this.fileSystemManager = new MasterFileSystem(this, this);
708 
709     // enable table descriptors cache
710     this.tableDescriptors.setCacheOn();
711     // set the META's descriptor to the correct replication
712     this.tableDescriptors.get(TableName.META_TABLE_NAME).setRegionReplication(
713         conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM));
714     // warm-up HTDs cache on master initialization
715     if (preLoadTableDescriptors) {
716       status.setStatus("Pre-loading table descriptors");
717       this.tableDescriptors.getAll();
718     }
719 
720     // publish cluster ID
721     status.setStatus("Publishing Cluster ID in ZooKeeper");
722     ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
723     this.initLatch.countDown();
724     this.serverManager = createServerManager(this, this);
725 
726     setupClusterConnection();
727 
728     // Invalidate all write locks held previously
729     this.tableLockManager.reapWriteLocks();
730 
731     status.setStatus("Initializing ZK system trackers");
732     initializeZKBasedSystemTrackers();
733 
734     // initialize master side coprocessors before we start handling requests
735     status.setStatus("Initializing master coprocessors");
736     this.cpHost = new MasterCoprocessorHost(this, this.conf);
737 
738     // start up all service threads.
739     status.setStatus("Initializing master service threads");
740     startServiceThreads();
741 
742     // Wake up this server to check in
743     sleeper.skipSleepCycle();
744 
745     // Wait for region servers to report in
746     this.serverManager.waitForRegionServers(status);
747     // Check zk for region servers that are up but didn't register
748     for (ServerName sn: this.regionServerTracker.getOnlineServers()) {
749       // The isServerOnline check is opportunistic, correctness is handled inside
750       if (!this.serverManager.isServerOnline(sn)
751           && serverManager.checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
752         LOG.info("Registered server found up in zk but who has not yet reported in: " + sn);
753       }
754     }
755 
756     // get a list for previously failed RS which need log splitting work
757     // we recover hbase:meta region servers inside master initialization and
758     // handle other failed servers in SSH in order to start up master node ASAP
759     Set<ServerName> previouslyFailedServers = this.fileSystemManager
760         .getFailedServersFromLogFolders();
761 
762     // remove stale recovering regions from previous run
763     this.fileSystemManager.removeStaleRecoveringRegionsFromZK(previouslyFailedServers);
764 
765     // log splitting for hbase:meta server
766     ServerName oldMetaServerLocation = metaTableLocator.getMetaRegionLocation(this.getZooKeeper());
767     if (oldMetaServerLocation != null && previouslyFailedServers.contains(oldMetaServerLocation)) {
768       splitMetaLogBeforeAssignment(oldMetaServerLocation);
769       // Note: we can't remove oldMetaServerLocation from previousFailedServers list because it
770       // may also host user regions
771     }
772     Set<ServerName> previouslyFailedMetaRSs = getPreviouselyFailedMetaServersFromZK();
773     // need to use union of previouslyFailedMetaRSs recorded in ZK and previouslyFailedServers
774     // instead of previouslyFailedMetaRSs alone to address the following two situations:
775     // 1) the chained failure situation(recovery failed multiple times in a row).
776     // 2) master get killed right before it could delete the recovering hbase:meta from ZK while the
777     // same server still has non-meta wals to be replayed so that
778     // removeStaleRecoveringRegionsFromZK can't delete the stale hbase:meta region
779     // Passing more servers into splitMetaLog is all right. If a server doesn't have hbase:meta wal,
780     // there is no op for the server.
781     previouslyFailedMetaRSs.addAll(previouslyFailedServers);
782 
783     this.initializationBeforeMetaAssignment = true;
784 
785     // Wait for regionserver to finish initialization.
786     if (BaseLoadBalancer.tablesOnMaster(conf)) {
787       waitForServerOnline();
788     }
789 
790     //initialize load balancer
791     this.balancer.setClusterStatus(getClusterStatus());
792     this.balancer.setMasterServices(this);
793     this.balancer.initialize();
794 
795     // Check if master is shutting down because of some issue
796     // in initializing the regionserver or the balancer.
797     if(isStopped()) return;
798 
799     // Make sure meta assigned before proceeding.
800     status.setStatus("Assigning Meta Region");
801     assignMeta(status, previouslyFailedMetaRSs, HRegionInfo.DEFAULT_REPLICA_ID);
802     // check if master is shutting down because above assignMeta could return even hbase:meta isn't
803     // assigned when master is shutting down
804     if(isStopped()) return;
805 
806     status.setStatus("Submitting log splitting work for previously failed region servers");
807     // Master has recovered hbase:meta region server and we put
808     // other failed region servers in a queue to be handled later by SSH
809     for (ServerName tmpServer : previouslyFailedServers) {
810       this.serverManager.processDeadServer(tmpServer, true);
811     }
812 
813     // Update meta with new PB serialization if required. i.e migrate all HRI to PB serialization
814     // in meta. This must happen before we assign all user regions or else the assignment will
815     // fail.
816     if (this.conf.getBoolean("hbase.MetaMigrationConvertingToPB", true)) {
817       MetaMigrationConvertingToPB.updateMetaIfNecessary(this);
818     }
819 
820     // Fix up assignment manager status
821     status.setStatus("Starting assignment manager");
822     this.assignmentManager.joinCluster();
823 
824     //set cluster status again after user regions are assigned
825     this.balancer.setClusterStatus(getClusterStatus());
826 
827     // Start balancer and meta catalog janitor after meta and regions have
828     // been assigned.
829     status.setStatus("Starting balancer and catalog janitor");
830     this.clusterStatusChore = new ClusterStatusChore(this, balancer);
831     getChoreService().scheduleChore(clusterStatusChore);
832     this.balancerChore = new BalancerChore(this);
833     getChoreService().scheduleChore(balancerChore);
834     this.normalizerChore = new RegionNormalizerChore(this);
835     getChoreService().scheduleChore(normalizerChore);
836     this.catalogJanitorChore = new CatalogJanitor(this, this);
837     getChoreService().scheduleChore(catalogJanitorChore);
838 
839     status.setStatus("Starting namespace manager");
840     initNamespace();
841 
842     if (this.cpHost != null) {
843       try {
844         this.cpHost.preMasterInitialization();
845       } catch (IOException e) {
846         LOG.error("Coprocessor preMasterInitialization() hook failed", e);
847       }
848     }
849 
850     status.markComplete("Initialization successful");
851     LOG.info("Master has completed initialization");
852     configurationManager.registerObserver(this.balancer);
853     initialized = true;
854 
855     status.setStatus("Starting quota manager");
856     initQuotaManager();
857 
858     // assign the meta replicas
859     Set<ServerName> EMPTY_SET = new HashSet<ServerName>();
860     int numReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
861            HConstants.DEFAULT_META_REPLICA_NUM);
862     for (int i = 1; i < numReplicas; i++) {
863       assignMeta(status, EMPTY_SET, i);
864     }
865     unassignExcessMetaReplica(zooKeeper, numReplicas);
866 
867     // clear the dead servers with same host name and port of online server because we are not
868     // removing dead server with same hostname and port of rs which is trying to check in before
869     // master initialization. See HBASE-5916.
870     this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
871 
872     // Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
873     status.setStatus("Checking ZNode ACLs");
874     zooKeeper.checkAndSetZNodeAcls();
875 
876     status.setStatus("Calling postStartMaster coprocessors");
877 
878     this.expiredMobFileCleanerChore = new ExpiredMobFileCleanerChore(this);
879     getChoreService().scheduleChore(expiredMobFileCleanerChore);
880 
881     int mobCompactionPeriod = conf.getInt(MobConstants.MOB_COMPACTION_CHORE_PERIOD,
882       MobConstants.DEFAULT_MOB_COMPACTION_CHORE_PERIOD);
883     if (mobCompactionPeriod > 0) {
884       this.mobCompactChore = new MobCompactionChore(this, mobCompactionPeriod);
885       getChoreService().scheduleChore(mobCompactChore);
886     } else {
887       LOG
888         .info("The period is " + mobCompactionPeriod + " seconds, MobCompactionChore is disabled");
889     }
890     this.mobCompactThread = new MasterMobCompactionThread(this);
891 
892     if (this.cpHost != null) {
893       // don't let cp initialization errors kill the master
894       try {
895         this.cpHost.postStartMaster();
896       } catch (IOException ioe) {
897         LOG.error("Coprocessor postStartMaster() hook failed", ioe);
898       }
899     }
900 
901     zombieDetector.interrupt();
902   }
903 
904   private void initQuotaManager() throws IOException {
905     quotaManager = new MasterQuotaManager(this);
906     this.assignmentManager.setRegionStateListener((RegionStateListener) quotaManager);
907     quotaManager.start();
908   }
909 
910   /**
911    * Create a {@link ServerManager} instance.
912    * @param master
913    * @param services
914    * @return An instance of {@link ServerManager}
915    * @throws org.apache.hadoop.hbase.ZooKeeperConnectionException
916    * @throws IOException
917    */
918   ServerManager createServerManager(final Server master,
919       final MasterServices services)
920   throws IOException {
921     // We put this out here in a method so can do a Mockito.spy and stub it out
922     // w/ a mocked up ServerManager.
923     return new ServerManager(master, services);
924   }
925 
926   private void unassignExcessMetaReplica(ZooKeeperWatcher zkw, int numMetaReplicasConfigured) {
927     // unassign the unneeded replicas (for e.g., if the previous master was configured
928     // with a replication of 3 and now it is 2, we need to unassign the 1 unneeded replica)
929     try {
930       List<String> metaReplicaZnodes = zooKeeper.getMetaReplicaNodes();
931       for (String metaReplicaZnode : metaReplicaZnodes) {
932         int replicaId = zooKeeper.getMetaReplicaIdFromZnode(metaReplicaZnode);
933         if (replicaId >= numMetaReplicasConfigured) {
934           RegionState r = MetaTableLocator.getMetaRegionState(zkw, replicaId);
935           LOG.info("Closing excess replica of meta region " + r.getRegion());
936           // send a close and wait for a max of 30 seconds
937           ServerManager.closeRegionSilentlyAndWait(getConnection(), r.getServerName(),
938               r.getRegion(), 30000);
939           ZKUtil.deleteNode(zkw, zkw.getZNodeForReplica(replicaId));
940         }
941       }
942     } catch (Exception ex) {
943       // ignore the exception since we don't want the master to be wedged due to potential
944       // issues in the cleanup of the extra regions. We can do that cleanup via hbck or manually
945       LOG.warn("Ignoring exception " + ex);
946     }
947   }
948 
949   /**
950    * Check <code>hbase:meta</code> is assigned. If not, assign it.
951    * @param status MonitoredTask
952    * @param previouslyFailedMetaRSs
953    * @param replicaId
954    * @throws InterruptedException
955    * @throws IOException
956    * @throws KeeperException
957    */
958   void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMetaRSs, int replicaId)
959       throws InterruptedException, IOException, KeeperException {
960     // Work on meta region
961     int assigned = 0;
962     long timeout = this.conf.getLong("hbase.catalog.verification.timeout", 1000);
963     if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
964       status.setStatus("Assigning hbase:meta region");
965     } else {
966       status.setStatus("Assigning hbase:meta region, replicaId " + replicaId);
967     }
968     // Get current meta state from zk.
969     RegionStates regionStates = assignmentManager.getRegionStates();
970     RegionState metaState = MetaTableLocator.getMetaRegionState(getZooKeeper(), replicaId);
971     HRegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO,
972         replicaId);
973     ServerName currentMetaServer = metaState.getServerName();
974     if (!ConfigUtil.useZKForAssignment(conf)) {
975       regionStates.createRegionState(hri, metaState.getState(),
976         currentMetaServer, null);
977     } else {
978       regionStates.createRegionState(hri);
979     }
980     boolean rit = this.assignmentManager.
981       processRegionInTransitionAndBlockUntilAssigned(hri);
982     boolean metaRegionLocation = metaTableLocator.verifyMetaRegionLocation(
983       this.getConnection(), this.getZooKeeper(), timeout, replicaId);
984     if (!metaRegionLocation || !metaState.isOpened()) {
985       // Meta location is not verified. It should be in transition, or offline.
986       // We will wait for it to be assigned in enableSSHandWaitForMeta below.
987       assigned++;
988       if (!ConfigUtil.useZKForAssignment(conf)) {
989         assignMetaZkLess(regionStates, metaState, timeout, previouslyFailedMetaRSs);
990       } else if (!rit) {
991         // Assign meta since not already in transition
992         if (currentMetaServer != null) {
993           // If the meta server is not known to be dead or online,
994           // just split the meta log, and don't expire it since this
995           // could be a full cluster restart. Otherwise, we will think
996           // this is a failover and lose previous region locations.
997           // If it is really a failover case, AM will find out in rebuilding
998           // user regions. Otherwise, we are good since all logs are split
999           // or known to be replayed before user regions are assigned.
1000           if (serverManager.isServerOnline(currentMetaServer)) {
1001             LOG.info("Forcing expire of " + currentMetaServer);
1002             serverManager.expireServer(currentMetaServer);
1003           }
1004           if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) {
1005             splitMetaLogBeforeAssignment(currentMetaServer);
1006             previouslyFailedMetaRSs.add(currentMetaServer);
1007           }
1008         }
1009         assignmentManager.assignMeta(hri);
1010       }
1011     } else {
1012       // Region already assigned. We didn't assign it. Add to in-memory state.
1013       regionStates.updateRegionState(
1014         HRegionInfo.FIRST_META_REGIONINFO, State.OPEN, currentMetaServer);
1015       this.assignmentManager.regionOnline(
1016         HRegionInfo.FIRST_META_REGIONINFO, currentMetaServer);
1017     }
1018 
1019     if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableMeta(TableName.META_TABLE_NAME);
1020 
1021     if ((RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode())
1022         && (!previouslyFailedMetaRSs.isEmpty())) {
1023       // replay WAL edits mode need new hbase:meta RS is assigned firstly
1024       status.setStatus("replaying log for Meta Region");
1025       this.fileSystemManager.splitMetaLog(previouslyFailedMetaRSs);
1026     }
1027 
1028     // Make sure a hbase:meta location is set. We need to enable SSH here since
1029     // if the meta region server is died at this time, we need it to be re-assigned
1030     // by SSH so that system tables can be assigned.
1031     // No need to wait for meta is assigned = 0 when meta is just verified.
1032     if (replicaId == HRegionInfo.DEFAULT_REPLICA_ID) enableServerShutdownHandler(assigned != 0);
1033     LOG.info("hbase:meta with replicaId " + replicaId + " assigned=" + assigned + ", rit=" + rit +
1034       ", location=" + metaTableLocator.getMetaRegionLocation(this.getZooKeeper(), replicaId));
1035     status.setStatus("META assigned.");
1036   }
1037 
1038   private void assignMetaZkLess(RegionStates regionStates, RegionState regionState, long timeout,
1039       Set<ServerName> previouslyFailedRs) throws IOException, KeeperException {
1040     ServerName currentServer = regionState.getServerName();
1041     if (serverManager.isServerOnline(currentServer)) {
1042       LOG.info("Meta was in transition on " + currentServer);
1043       assignmentManager.processRegionInTransitionZkLess();
1044     } else {
1045       if (currentServer != null) {
1046         if (regionState.getRegion().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
1047           splitMetaLogBeforeAssignment(currentServer);
1048           regionStates.logSplit(HRegionInfo.FIRST_META_REGIONINFO);
1049           previouslyFailedRs.add(currentServer);
1050         }
1051       }
1052       LOG.info("Re-assigning hbase:meta, it was on " + currentServer);
1053       regionStates.updateRegionState(regionState.getRegion(), State.OFFLINE);
1054       assignmentManager.assignMeta(regionState.getRegion());
1055     }
1056   }
1057 
1058   void initNamespace() throws IOException {
1059     //create namespace manager
1060     tableNamespaceManager = new TableNamespaceManager(this);
1061     tableNamespaceManager.start();
1062   }
1063 
1064   boolean isCatalogJanitorEnabled() {
1065     return catalogJanitorChore != null ?
1066       catalogJanitorChore.getEnabled() : false;
1067   }
1068 
1069   private void splitMetaLogBeforeAssignment(ServerName currentMetaServer) throws IOException {
1070     if (RecoveryMode.LOG_REPLAY == this.getMasterFileSystem().getLogRecoveryMode()) {
1071       // In log replay mode, we mark hbase:meta region as recovering in ZK
1072       Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
1073       regions.add(HRegionInfo.FIRST_META_REGIONINFO);
1074       this.fileSystemManager.prepareLogReplay(currentMetaServer, regions);
1075     } else {
1076       // In recovered.edits mode: create recovered edits file for hbase:meta server
1077       this.fileSystemManager.splitMetaLog(currentMetaServer);
1078     }
1079   }
1080 
1081   private void enableServerShutdownHandler(
1082       final boolean waitForMeta) throws IOException, InterruptedException {
1083     // If ServerShutdownHandler is disabled, we enable it and expire those dead
1084     // but not expired servers. This is required so that if meta is assigning to
1085     // a server which dies after assignMeta starts assignment,
1086     // SSH can re-assign it. Otherwise, we will be
1087     // stuck here waiting forever if waitForMeta is specified.
1088     if (!serverShutdownHandlerEnabled) {
1089       serverShutdownHandlerEnabled = true;
1090       this.serverManager.processQueuedDeadServers();
1091     }
1092 
1093     if (waitForMeta) {
1094       metaTableLocator.waitMetaRegionLocation(this.getZooKeeper());
1095       // Above check waits for general meta availability but this does not
1096       // guarantee that the transition has completed
1097       this.assignmentManager.waitForAssignment(HRegionInfo.FIRST_META_REGIONINFO);
1098     }
1099   }
1100 
1101   private void enableMeta(TableName metaTableName) {
1102     if (!this.assignmentManager.getTableStateManager().isTableState(metaTableName,
1103         ZooKeeperProtos.Table.State.ENABLED)) {
1104       this.assignmentManager.setEnabledTable(metaTableName);
1105     }
1106   }
1107 
1108   /**
1109    * This function returns a set of region server names under hbase:meta recovering region ZK node
1110    * @return Set of meta server names which were recorded in ZK
1111    * @throws KeeperException
1112    */
1113   private Set<ServerName> getPreviouselyFailedMetaServersFromZK() throws KeeperException {
1114     Set<ServerName> result = new HashSet<ServerName>();
1115     String metaRecoveringZNode = ZKUtil.joinZNode(zooKeeper.recoveringRegionsZNode,
1116       HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
1117     List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(zooKeeper, metaRecoveringZNode);
1118     if (regionFailedServers == null) return result;
1119 
1120     for(String failedServer : regionFailedServers) {
1121       ServerName server = ServerName.parseServerName(failedServer);
1122       result.add(server);
1123     }
1124     return result;
1125   }
1126 
1127   @Override
1128   public TableDescriptors getTableDescriptors() {
1129     return this.tableDescriptors;
1130   }
1131 
1132   @Override
1133   public ServerManager getServerManager() {
1134     return this.serverManager;
1135   }
1136 
1137   @Override
1138   public MasterFileSystem getMasterFileSystem() {
1139     return this.fileSystemManager;
1140   }
1141 
1142   /*
1143    * Start up all services. If any of these threads gets an unhandled exception
1144    * then they just die with a logged message.  This should be fine because
1145    * in general, we do not expect the master to get such unhandled exceptions
1146    *  as OOMEs; it should be lightly loaded. See what HRegionServer does if
1147    *  need to install an unexpected exception handler.
1148    */
1149   private void startServiceThreads() throws IOException{
1150    // Start the executor service pools
1151    this.service.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
1152       conf.getInt("hbase.master.executor.openregion.threads", 5));
1153    this.service.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
1154       conf.getInt("hbase.master.executor.closeregion.threads", 5));
1155    this.service.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
1156       conf.getInt("hbase.master.executor.serverops.threads", 5));
1157    this.service.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
1158       conf.getInt("hbase.master.executor.serverops.threads", 5));
1159    this.service.startExecutorService(ExecutorType.M_LOG_REPLAY_OPS,
1160       conf.getInt("hbase.master.executor.logreplayops.threads", 10));
1161 
1162    // We depend on there being only one instance of this executor running
1163    // at a time.  To do concurrency, would need fencing of enable/disable of
1164    // tables.
1165    // Any time changing this maxThreads to > 1, pls see the comment at
1166    // AccessController#postCreateTableHandler
1167    this.service.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
1168    startProcedureExecutor();
1169 
1170    // Start log cleaner thread
1171    int cleanerInterval = conf.getInt("hbase.master.cleaner.interval", 60 * 1000);
1172    this.logCleaner =
1173       new LogCleaner(cleanerInterval,
1174          this, conf, getMasterFileSystem().getFileSystem(),
1175          getMasterFileSystem().getOldLogDir());
1176     getChoreService().scheduleChore(logCleaner);
1177 
1178    //start the hfile archive cleaner thread
1179     Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
1180     this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem()
1181         .getFileSystem(), archiveDir);
1182     getChoreService().scheduleChore(hfileCleaner);
1183     serviceStarted = true;
1184     if (LOG.isTraceEnabled()) {
1185       LOG.trace("Started service threads");
1186     }
1187   }
1188 
1189   @Override
1190   protected void sendShutdownInterrupt() {
1191     super.sendShutdownInterrupt();
1192     stopProcedureExecutor();
1193   }
1194 
1195   @Override
1196   protected void stopServiceThreads() {
1197     if (masterJettyServer != null) {
1198       LOG.info("Stopping master jetty server");
1199       try {
1200         masterJettyServer.stop();
1201       } catch (Exception e) {
1202         LOG.error("Failed to stop master jetty server", e);
1203       }
1204     }
1205     super.stopServiceThreads();
1206     stopChores();
1207 
1208     // Wait for all the remaining region servers to report in IFF we were
1209     // running a cluster shutdown AND we were NOT aborting.
1210     if (!isAborted() && this.serverManager != null &&
1211         this.serverManager.isClusterShutdown()) {
1212       this.serverManager.letRegionServersShutdown();
1213     }
1214     if (LOG.isDebugEnabled()) {
1215       LOG.debug("Stopping service threads");
1216     }
1217     // Clean up and close up shop
1218     if (this.logCleaner != null) this.logCleaner.cancel(true);
1219     if (this.hfileCleaner != null) this.hfileCleaner.cancel(true);
1220     if (this.quotaManager != null) this.quotaManager.stop();
1221     if (this.activeMasterManager != null) this.activeMasterManager.stop();
1222     if (this.serverManager != null) this.serverManager.stop();
1223     if (this.assignmentManager != null) this.assignmentManager.stop();
1224     if (this.fileSystemManager != null) this.fileSystemManager.stop();
1225     if (this.mpmHost != null) this.mpmHost.stop("server shutting down.");
1226   }
1227 
1228   /**
1229    * Check whether the procedure executor is enabled
1230    */
1231   @Override
1232   public boolean isMasterProcedureExecutorEnabled() {
1233     return (this.procedureConf == ProcedureConf.PROCEDURE_ENABLED);
1234   }
1235 
1236   private void startProcedureExecutor() throws IOException {
1237     final MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
1238     final Path logDir = new Path(fileSystemManager.getRootDir(),
1239         MasterProcedureConstants.MASTER_PROCEDURE_LOGDIR);
1240 
1241     if (this.procedureConf == ProcedureConf.PROCEDURE_FULLY_DISABLED) {
1242       // Clean up the procedure store so that we will in a clean state when procedure
1243       // is enabled later.
1244       // Note: hbck might needed for uncompleted procedures.
1245       try {
1246         fs.delete(logDir, true);
1247         LOG.warn("Procedure executor is disabled from configuartion. " +
1248             "All the state logs from procedure store were removed." +
1249             "You should check the cluster state using HBCK.");
1250       } catch (Exception e) {
1251         // Ignore exception and move on.
1252         LOG.error("Removing all the state logs from procedure store failed." +
1253             "You should check the cluster state using HBCK.");
1254       }
1255       return;
1256     }
1257 
1258     procedureStore = new WALProcedureStore(conf, fileSystemManager.getFileSystem(), logDir,
1259         new MasterProcedureEnv.WALStoreLeaseRecovery(this));
1260 
1261     procedureStore.registerListener(new MasterProcedureEnv.MasterProcedureStoreListener(this));
1262     procedureExecutor = new ProcedureExecutor(conf, procEnv, procedureStore,
1263         procEnv.getProcedureQueue());
1264 
1265     final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS,
1266         Math.max(Runtime.getRuntime().availableProcessors(),
1267           MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
1268     procedureStore.start(numThreads);
1269     procedureExecutor.start(numThreads);
1270   }
1271 
1272   private void stopProcedureExecutor() {
1273     if (procedureExecutor != null) {
1274       procedureExecutor.stop();
1275     }
1276 
1277     if (procedureStore != null) {
1278       procedureStore.stop(isAborted());
1279     }
1280   }
1281 
1282   private void stopChores() {
1283     if (this.expiredMobFileCleanerChore != null) {
1284       this.expiredMobFileCleanerChore.cancel(true);
1285     }
1286     if (this.mobCompactChore != null) {
1287       this.mobCompactChore.cancel(true);
1288     }
1289     if (this.balancerChore != null) {
1290       this.balancerChore.cancel(true);
1291     }
1292     if (this.normalizerChore != null) {
1293       this.normalizerChore.cancel(true);
1294     }
1295     if (this.clusterStatusChore != null) {
1296       this.clusterStatusChore.cancel(true);
1297     }
1298     if (this.catalogJanitorChore != null) {
1299       this.catalogJanitorChore.cancel(true);
1300     }
1301     if (this.clusterStatusPublisherChore != null){
1302       clusterStatusPublisherChore.cancel(true);
1303     }
1304     if (this.mobCompactThread != null) {
1305       this.mobCompactThread.close();
1306     }
1307   }
1308 
1309   /**
1310    * @return Get remote side's InetAddress
1311    * @throws UnknownHostException
1312    */
1313   InetAddress getRemoteInetAddress(final int port,
1314       final long serverStartCode) throws UnknownHostException {
1315     // Do it out here in its own little method so can fake an address when
1316     // mocking up in tests.
1317     InetAddress ia = RpcServer.getRemoteIp();
1318 
1319     // The call could be from the local regionserver,
1320     // in which case, there is no remote address.
1321     if (ia == null && serverStartCode == startcode) {
1322       InetSocketAddress isa = rpcServices.getSocketAddress();
1323       if (isa != null && isa.getPort() == port) {
1324         ia = isa.getAddress();
1325       }
1326     }
1327     return ia;
1328   }
1329 
1330   /**
1331    * @return Maximum time we should run balancer for
1332    */
1333   private int getBalancerCutoffTime() {
1334     int balancerCutoffTime =
1335       getConfiguration().getInt("hbase.balancer.max.balancing", -1);
1336     if (balancerCutoffTime == -1) {
1337       // No time period set so create one
1338       int balancerPeriod =
1339         getConfiguration().getInt("hbase.balancer.period", 300000);
1340       balancerCutoffTime = balancerPeriod;
1341       // If nonsense period, set it to balancerPeriod
1342       if (balancerCutoffTime <= 0) balancerCutoffTime = balancerPeriod;
1343     }
1344     return balancerCutoffTime;
1345   }
1346 
1347   public boolean balance() throws IOException {
1348     return balance(false);
1349   }
1350 
1351   public boolean balance(boolean force) throws IOException {
1352     // if master not initialized, don't run balancer.
1353     if (!this.initialized) {
1354       LOG.debug("Master has not been initialized, don't run balancer.");
1355       return false;
1356     }
1357     // Do this call outside of synchronized block.
1358     int maximumBalanceTime = getBalancerCutoffTime();
1359     synchronized (this.balancer) {
1360       // If balance not true, don't run balancer.
1361       if (!this.loadBalancerTracker.isBalancerOn()) return false;
1362       // Only allow one balance run at at time.
1363       if (this.assignmentManager.getRegionStates().isRegionsInTransition()) {
1364         Map<String, RegionState> regionsInTransition =
1365           this.assignmentManager.getRegionStates().getRegionsInTransition();
1366         // if hbase:meta region is in transition, result of assignment cannot be recorded
1367         // ignore the force flag in that case
1368         boolean metaInTransition = assignmentManager.getRegionStates().isMetaRegionInTransition();
1369         String prefix = force && !metaInTransition ? "R" : "Not r";
1370         LOG.debug(prefix + "unning balancer because " + regionsInTransition.size() +
1371           " region(s) in transition: " + org.apache.commons.lang.StringUtils.
1372             abbreviate(regionsInTransition.toString(), 256));
1373         if (!force || metaInTransition) return false;
1374       }
1375       if (this.serverManager.areDeadServersInProgress()) {
1376         LOG.debug("Not running balancer because processing dead regionserver(s): " +
1377           this.serverManager.getDeadServers());
1378         return false;
1379       }
1380 
1381       if (this.cpHost != null) {
1382         try {
1383           if (this.cpHost.preBalance()) {
1384             LOG.debug("Coprocessor bypassing balancer request");
1385             return false;
1386           }
1387         } catch (IOException ioe) {
1388           LOG.error("Error invoking master coprocessor preBalance()", ioe);
1389           return false;
1390         }
1391       }
1392 
1393       Map<TableName, Map<ServerName, List<HRegionInfo>>> assignmentsByTable =
1394         this.assignmentManager.getRegionStates().getAssignmentsByTable();
1395 
1396       List<RegionPlan> plans = new ArrayList<RegionPlan>();
1397 
1398       //Give the balancer the current cluster state.
1399       this.balancer.setClusterStatus(getClusterStatus());
1400       for (Entry<TableName, Map<ServerName, List<HRegionInfo>>> e : assignmentsByTable.entrySet()) {
1401         List<RegionPlan> partialPlans = this.balancer.balanceCluster(e.getKey(), e.getValue());
1402         if (partialPlans != null) plans.addAll(partialPlans);
1403       }
1404 
1405       long cutoffTime = System.currentTimeMillis() + maximumBalanceTime;
1406       int rpCount = 0;  // number of RegionPlans balanced so far
1407       long totalRegPlanExecTime = 0;
1408       if (plans != null && !plans.isEmpty()) {
1409         for (RegionPlan plan: plans) {
1410           LOG.info("balance " + plan);
1411           long balStartTime = System.currentTimeMillis();
1412           //TODO: bulk assign
1413           this.assignmentManager.balance(plan);
1414           totalRegPlanExecTime += System.currentTimeMillis()-balStartTime;
1415           rpCount++;
1416           if (rpCount < plans.size() &&
1417               // if performing next balance exceeds cutoff time, exit the loop
1418               (System.currentTimeMillis() + (totalRegPlanExecTime / rpCount)) > cutoffTime) {
1419             //TODO: After balance, there should not be a cutoff time (keeping it as a security net for now)
1420             LOG.debug("No more balancing till next balance run; maximumBalanceTime=" +
1421               maximumBalanceTime);
1422             break;
1423           }
1424         }
1425       }
1426       if (this.cpHost != null) {
1427         try {
1428           this.cpHost.postBalance(rpCount < plans.size() ? plans.subList(0, rpCount) : plans);
1429         } catch (IOException ioe) {
1430           // balancing already succeeded so don't change the result
1431           LOG.error("Error invoking master coprocessor postBalance()", ioe);
1432         }
1433       }
1434     }
1435     // If LoadBalancer did not generate any plans, it means the cluster is already balanced.
1436     // Return true indicating a success.
1437     return true;
1438   }
1439 
1440   /**
1441    * Perform normalization of cluster (invoked by {@link RegionNormalizerChore}).
1442    *
1443    * @return true if normalization step was performed successfully, false otherwise
1444    *   (specifically, if HMaster hasn't been initialized properly or normalization
1445    *   is globally disabled)
1446    * @throws IOException, CoordinatedStateException
1447    */
1448   public boolean normalizeRegions() throws IOException, CoordinatedStateException {
1449     if (!this.initialized) {
1450       LOG.debug("Master has not been initialized, don't run region normalizer.");
1451       return false;
1452     }
1453 
1454     if (!this.regionNormalizerTracker.isNormalizerOn()) {
1455       LOG.debug("Region normalization is disabled, don't run region normalizer.");
1456       return false;
1457     }
1458 
1459     synchronized (this.normalizer) {
1460       // Don't run the normalizer concurrently
1461       List<TableName> allEnabledTables = new ArrayList<>(
1462         this.assignmentManager.getTableStateManager().getTablesInStates(
1463           ZooKeeperProtos.Table.State.ENABLED));
1464 
1465       Collections.shuffle(allEnabledTables);
1466 
1467       for (TableName table : allEnabledTables) {
1468         if (table.isSystemTable() || (getTableDescriptors().get(table) != null &&
1469             !getTableDescriptors().get(table).isNormalizationEnabled())) {
1470           LOG.debug("Skipping normalization for table: " + table + ", as it's either system"
1471               + " table or doesn't have auto normalization turned on");
1472           continue;
1473         }
1474         List<NormalizationPlan> plans = this.normalizer.computePlanForTable(table);
1475         if (plans != null) {
1476           for (NormalizationPlan plan : plans) {
1477             plan.execute(clusterConnection.getAdmin());
1478           }
1479         }
1480       }
1481     }
1482     // If Region did not generate any plans, it means the cluster is already balanced.
1483     // Return true indicating a success.
1484     return true;
1485   }
1486 
1487   /**
1488    * @return Client info for use as prefix on an audit log string; who did an action
1489    */
1490   String getClientIdAuditPrefix() {
1491     return "Client=" + RpcServer.getRequestUserName() + "/" + RpcServer.getRemoteAddress();
1492   }
1493 
1494   /**
1495    * Switch for the background CatalogJanitor thread.
1496    * Used for testing.  The thread will continue to run.  It will just be a noop
1497    * if disabled.
1498    * @param b If false, the catalog janitor won't do anything.
1499    */
1500   public void setCatalogJanitorEnabled(final boolean b) {
1501     this.catalogJanitorChore.setEnabled(b);
1502   }
1503 
1504   @Override
1505   public void dispatchMergingRegions(final HRegionInfo region_a,
1506       final HRegionInfo region_b, final boolean forcible) throws IOException {
1507     checkInitialized();
1508     this.service.submit(new DispatchMergingRegionHandler(this,
1509       this.catalogJanitorChore, region_a, region_b, forcible));
1510   }
1511 
1512   void move(final byte[] encodedRegionName,
1513       final byte[] destServerName) throws HBaseIOException {
1514     RegionState regionState = assignmentManager.getRegionStates().
1515       getRegionState(Bytes.toString(encodedRegionName));
1516 
1517     HRegionInfo hri;
1518     if (regionState != null) {
1519       hri = regionState.getRegion();
1520     } else {
1521       throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
1522     }
1523 
1524     ServerName dest;
1525     if (destServerName == null || destServerName.length == 0) {
1526       LOG.info("Passed destination servername is null/empty so " +
1527         "choosing a server at random");
1528       final List<ServerName> destServers = this.serverManager.createDestinationServersList(
1529         regionState.getServerName());
1530       dest = balancer.randomAssignment(hri, destServers);
1531       if (dest == null) {
1532         LOG.debug("Unable to determine a plan to assign " + hri);
1533         return;
1534       }
1535     } else {
1536       ServerName candidate = ServerName.valueOf(Bytes.toString(destServerName));
1537       dest = balancer.randomAssignment(hri, Lists.newArrayList(candidate));
1538       if (dest == null) {
1539         LOG.debug("Unable to determine a plan to assign " + hri);
1540         return;
1541       }
1542       if (dest.equals(serverName) && balancer instanceof BaseLoadBalancer
1543           && !((BaseLoadBalancer)balancer).shouldBeOnMaster(hri)) {
1544         // To avoid unnecessary region moving later by balancer. Don't put user
1545         // regions on master. Regions on master could be put on other region
1546         // server intentionally by test however.
1547         LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
1548           + " to avoid unnecessary region moving later by load balancer,"
1549           + " because it should not be on master");
1550         return;
1551       }
1552     }
1553 
1554     if (dest.equals(regionState.getServerName())) {
1555       LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
1556         + " because region already assigned to the same server " + dest + ".");
1557       return;
1558     }
1559 
1560     // Now we can do the move
1561     RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);
1562 
1563     try {
1564       checkInitialized();
1565       if (this.cpHost != null) {
1566         if (this.cpHost.preMove(hri, rp.getSource(), rp.getDestination())) {
1567           return;
1568         }
1569       }
1570       // warmup the region on the destination before initiating the move. this call
1571       // is synchronous and takes some time. doing it before the source region gets
1572       // closed
1573       serverManager.sendRegionWarmup(rp.getDestination(), hri);
1574 
1575       LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
1576       this.assignmentManager.balance(rp);
1577       if (this.cpHost != null) {
1578         this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
1579       }
1580     } catch (IOException ioe) {
1581       if (ioe instanceof HBaseIOException) {
1582         throw (HBaseIOException)ioe;
1583       }
1584       throw new HBaseIOException(ioe);
1585     }
1586   }
1587 
1588   @Override
1589   public long createTable(HTableDescriptor hTableDescriptor,
1590       byte [][] splitKeys) throws IOException {
1591     if (isStopped()) {
1592       throw new MasterNotRunningException();
1593     }
1594 
1595     TableName tableName = hTableDescriptor.getTableName();
1596     String namespace = tableName.getNamespaceAsString();
1597     ensureNamespaceExists(namespace);
1598 
1599     HRegionInfo[] newRegions = ModifyRegionUtils.createHRegionInfos(hTableDescriptor, splitKeys);
1600     checkInitialized();
1601     sanityCheckTableDescriptor(hTableDescriptor);
1602     if (cpHost != null) {
1603       cpHost.preCreateTable(hTableDescriptor, newRegions);
1604     }
1605     LOG.info(getClientIdAuditPrefix() + " create " + hTableDescriptor);
1606 
1607     long procId = -1;
1608     if (isMasterProcedureExecutorEnabled()) {
1609       // TODO: We can handle/merge duplicate requests, and differentiate the case of
1610       //       TableExistsException by saying if the schema is the same or not.
1611       ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
1612       procId = this.procedureExecutor.submitProcedure(
1613         new CreateTableProcedure(procedureExecutor.getEnvironment(),
1614           hTableDescriptor, newRegions, latch));
1615       latch.await();
1616     } else {
1617       try {
1618         this.quotaManager.checkNamespaceTableAndRegionQuota(tableName, newRegions.length);
1619         this.service.submit(new CreateTableHandler(this, this.fileSystemManager, hTableDescriptor,
1620             conf, newRegions, this).prepare());
1621       } catch (IOException e) {
1622         this.quotaManager.removeTableFromNamespaceQuota(tableName);
1623         LOG.error("Exception occurred while creating the table " + tableName.getNameAsString(), e);
1624         throw e;
1625       }
1626     }
1627 
1628     if (cpHost != null) {
1629       cpHost.postCreateTable(hTableDescriptor, newRegions);
1630     }
1631 
1632     return procId;
1633   }
1634 
1635   /**
1636    * Checks whether the table conforms to some sane limits, and configured
1637    * values (compression, etc) work. Throws an exception if something is wrong.
1638    * @throws IOException
1639    */
1640   private void sanityCheckTableDescriptor(final HTableDescriptor htd) throws IOException {
1641     final String CONF_KEY = "hbase.table.sanity.checks";
1642     boolean logWarn = false;
1643     if (!conf.getBoolean(CONF_KEY, true)) {
1644       logWarn = true;
1645     }
1646     String tableVal = htd.getConfigurationValue(CONF_KEY);
1647     if (tableVal != null && !Boolean.valueOf(tableVal)) {
1648       logWarn = true;
1649     }
1650 
1651     // check max file size
1652     long maxFileSizeLowerLimit = 2 * 1024 * 1024L; // 2M is the default lower limit
1653     long maxFileSize = htd.getMaxFileSize();
1654     if (maxFileSize < 0) {
1655       maxFileSize = conf.getLong(HConstants.HREGION_MAX_FILESIZE, maxFileSizeLowerLimit);
1656     }
1657     if (maxFileSize < conf.getLong("hbase.hregion.max.filesize.limit", maxFileSizeLowerLimit)) {
1658       String message = "MAX_FILESIZE for table descriptor or "
1659           + "\"hbase.hregion.max.filesize\" (" + maxFileSize
1660           + ") is too small, which might cause over splitting into unmanageable "
1661           + "number of regions.";
1662       warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
1663     }
1664 
1665     // check flush size
1666     long flushSizeLowerLimit = 1024 * 1024L; // 1M is the default lower limit
1667     long flushSize = htd.getMemStoreFlushSize();
1668     if (flushSize < 0) {
1669       flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, flushSizeLowerLimit);
1670     }
1671     if (flushSize < conf.getLong("hbase.hregion.memstore.flush.size.limit", flushSizeLowerLimit)) {
1672       String message = "MEMSTORE_FLUSHSIZE for table descriptor or "
1673           + "\"hbase.hregion.memstore.flush.size\" ("+flushSize+") is too small, which might cause"
1674           + " very frequent flushing.";
1675       warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
1676     }
1677 
1678     // check that coprocessors and other specified plugin classes can be loaded
1679     try {
1680       checkClassLoading(conf, htd);
1681     } catch (Exception ex) {
1682       warnOrThrowExceptionForFailure(logWarn, CONF_KEY, ex.getMessage(), null);
1683     }
1684 
1685     // check compression can be loaded
1686     try {
1687       checkCompression(htd);
1688     } catch (IOException e) {
1689       warnOrThrowExceptionForFailure(logWarn, CONF_KEY, e.getMessage(), e);
1690     }
1691 
1692     // check encryption can be loaded
1693     try {
1694       checkEncryption(conf, htd);
1695     } catch (IOException e) {
1696       warnOrThrowExceptionForFailure(logWarn, CONF_KEY, e.getMessage(), e);
1697     }
1698     // Verify compaction policy
1699     try{
1700       checkCompactionPolicy(conf, htd);
1701     } catch(IOException e){
1702       warnOrThrowExceptionForFailure(false, CONF_KEY, e.getMessage(), e);
1703     }
1704     // check that we have at least 1 CF
1705     if (htd.getColumnFamilies().length == 0) {
1706       String message = "Table should have at least one column family.";
1707       warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
1708     }
1709 
1710     for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
1711       if (hcd.getTimeToLive() <= 0) {
1712         String message = "TTL for column family " + hcd.getNameAsString() + " must be positive.";
1713         warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
1714       }
1715 
1716       // check blockSize
1717       if (hcd.getBlocksize() < 1024 || hcd.getBlocksize() > 16 * 1024 * 1024) {
1718         String message = "Block size for column family " + hcd.getNameAsString()
1719             + "  must be between 1K and 16MB.";
1720         warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
1721       }
1722 
1723       // check versions
1724       if (hcd.getMinVersions() < 0) {
1725         String message = "Min versions for column family " + hcd.getNameAsString()
1726           + "  must be positive.";
1727         warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
1728       }
1729       // max versions already being checked
1730 
1731       // HBASE-13776 Setting illegal versions for HColumnDescriptor
1732       //  does not throw IllegalArgumentException
1733       // check minVersions <= maxVerions
1734       if (hcd.getMinVersions() > hcd.getMaxVersions()) {
1735         String message = "Min versions for column family " + hcd.getNameAsString()
1736             + " must be less than the Max versions.";
1737         warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
1738       }
1739 
1740       // check replication scope
1741       if (hcd.getScope() < 0) {
1742         String message = "Replication scope for column family "
1743           + hcd.getNameAsString() + "  must be positive.";
1744         warnOrThrowExceptionForFailure(logWarn, CONF_KEY, message, null);
1745       }
1746 
1747       // TODO: should we check coprocessors and encryption ?
1748     }
1749   }
1750 
1751   private void checkCompactionPolicy(Configuration conf, HTableDescriptor htd)
1752       throws IOException {
1753     // FIFO compaction has some requirements
1754     // Actually FCP ignores periodic major compactions
1755     String className =
1756         htd.getConfigurationValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY);
1757     if (className == null) {
1758       className =
1759           conf.get(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
1760             ExploringCompactionPolicy.class.getName());
1761     }
1762 
1763 
1764     int blockingFileCount = HStore.DEFAULT_BLOCKING_STOREFILE_COUNT;
1765     String sv = htd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY);
1766     if (sv != null) {
1767       blockingFileCount = Integer.parseInt(sv);
1768     } else {
1769       blockingFileCount = conf.getInt(HStore.BLOCKING_STOREFILES_KEY, blockingFileCount);
1770     }
1771 
1772     for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
1773       String compactionPolicy =
1774           hcd.getConfigurationValue(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY);
1775       if (compactionPolicy == null) {
1776         compactionPolicy = className;
1777       }
1778       if (compactionPolicy.equals(FIFOCompactionPolicy.class.getName()) == false) {
1779         continue;
1780       }
1781       // FIFOCompaction
1782       String message = null;
1783 
1784       // 1. Check TTL
1785       if (hcd.getTimeToLive() == HColumnDescriptor.DEFAULT_TTL) {
1786         message = "Default TTL is not supported for FIFO compaction";
1787         throw new IOException(message);
1788       }
1789 
1790       // 2. Check min versions
1791       if (hcd.getMinVersions() > 0) {
1792         message = "MIN_VERSION > 0 is not supported for FIFO compaction";
1793         throw new IOException(message);
1794       }
1795 
1796       // 3. blocking file count
1797       String sbfc = htd.getConfigurationValue(HStore.BLOCKING_STOREFILES_KEY);
1798       if (sbfc != null) {
1799         blockingFileCount = Integer.parseInt(sbfc);
1800       }
1801       if (blockingFileCount < 1000) {
1802         message =
1803             "blocking file count '" + HStore.BLOCKING_STOREFILES_KEY + "' " + blockingFileCount
1804                 + " is below recommended minimum of 1000";
1805         throw new IOException(message);
1806       }
1807     }
1808   }
1809 
1810   // HBASE-13350 - Helper method to log warning on sanity check failures if checks disabled.
1811   private static void warnOrThrowExceptionForFailure(boolean logWarn, String confKey,
1812       String message, Exception cause) throws IOException {
1813     if (!logWarn) {
1814       throw new DoNotRetryIOException(message + " Set " + confKey +
1815           " to false at conf or table descriptor if you want to bypass sanity checks", cause);
1816     }
1817     LOG.warn(message);
1818   }
1819 
1820   private void startActiveMasterManager(int infoPort) throws KeeperException {
1821     String backupZNode = ZKUtil.joinZNode(
1822       zooKeeper.backupMasterAddressesZNode, serverName.toString());
1823     /*
1824     * Add a ZNode for ourselves in the backup master directory since we
1825     * may not become the active master. If so, we want the actual active
1826     * master to know we are backup masters, so that it won't assign
1827     * regions to us if so configured.
1828     *
1829     * If we become the active master later, ActiveMasterManager will delete
1830     * this node explicitly.  If we crash before then, ZooKeeper will delete
1831     * this node for us since it is ephemeral.
1832     */
1833     LOG.info("Adding backup master ZNode " + backupZNode);
1834     if (!MasterAddressTracker.setMasterAddress(zooKeeper, backupZNode,
1835         serverName, infoPort)) {
1836       LOG.warn("Failed create of " + backupZNode + " by " + serverName);
1837     }
1838 
1839     activeMasterManager.setInfoPort(infoPort);
1840     // Start a thread to try to become the active master, so we won't block here
1841     Threads.setDaemonThreadRunning(new Thread(new Runnable() {
1842       @Override
1843       public void run() {
1844         int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
1845           HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
1846         // If we're a backup master, stall until a primary to writes his address
1847         if (conf.getBoolean(HConstants.MASTER_TYPE_BACKUP,
1848           HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
1849           LOG.debug("HMaster started in backup mode. "
1850             + "Stalling until master znode is written.");
1851           // This will only be a minute or so while the cluster starts up,
1852           // so don't worry about setting watches on the parent znode
1853           while (!activeMasterManager.hasActiveMaster()) {
1854             LOG.debug("Waiting for master address ZNode to be written "
1855               + "(Also watching cluster state node)");
1856             Threads.sleep(timeout);
1857           }
1858         }
1859         MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
1860         status.setDescription("Master startup");
1861         try {
1862           if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
1863             finishActiveMasterInitialization(status);
1864           }
1865         } catch (Throwable t) {
1866           status.setStatus("Failed to become active: " + t.getMessage());
1867           LOG.fatal("Failed to become active master", t);
1868           // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
1869           if (t instanceof NoClassDefFoundError &&
1870               t.getMessage().contains("org/apache/hadoop/hdfs/protocol/FSConstants$SafeModeAction")) {
1871             // improved error message for this special case
1872             abort("HBase is having a problem with its Hadoop jars.  You may need to "
1873               + "recompile HBase against Hadoop version "
1874               +  org.apache.hadoop.util.VersionInfo.getVersion()
1875               + " or change your hadoop jars to start properly", t);
1876           } else {
1877             abort("Unhandled exception. Starting shutdown.", t);
1878           }
1879         } finally {
1880           status.cleanup();
1881         }
1882       }
1883     }, getServerName().toShortString() + ".activeMasterManager"));
1884   }
1885 
1886   private void checkCompression(final HTableDescriptor htd)
1887   throws IOException {
1888     if (!this.masterCheckCompression) return;
1889     for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
1890       checkCompression(hcd);
1891     }
1892   }
1893 
1894   private void checkCompression(final HColumnDescriptor hcd)
1895   throws IOException {
1896     if (!this.masterCheckCompression) return;
1897     CompressionTest.testCompression(hcd.getCompression());
1898     CompressionTest.testCompression(hcd.getCompactionCompression());
1899   }
1900 
1901   private void checkEncryption(final Configuration conf, final HTableDescriptor htd)
1902   throws IOException {
1903     if (!this.masterCheckEncryption) return;
1904     for (HColumnDescriptor hcd : htd.getColumnFamilies()) {
1905       checkEncryption(conf, hcd);
1906     }
1907   }
1908 
1909   private void checkEncryption(final Configuration conf, final HColumnDescriptor hcd)
1910   throws IOException {
1911     if (!this.masterCheckEncryption) return;
1912     EncryptionTest.testEncryption(conf, hcd.getEncryptionType(), hcd.getEncryptionKey());
1913   }
1914 
1915   private void checkClassLoading(final Configuration conf, final HTableDescriptor htd)
1916   throws IOException {
1917     RegionSplitPolicy.getSplitPolicyClass(htd, conf);
1918     RegionCoprocessorHost.testTableCoprocessorAttrs(conf, htd);
1919   }
1920 
1921   private static boolean isCatalogTable(final TableName tableName) {
1922     return tableName.equals(TableName.META_TABLE_NAME);
1923   }
1924 
1925   @Override
1926   public long deleteTable(final TableName tableName) throws IOException {
1927     checkInitialized();
1928     if (cpHost != null) {
1929       cpHost.preDeleteTable(tableName);
1930     }
1931     LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
1932 
1933     long procId = -1;
1934     if (isMasterProcedureExecutorEnabled()) {
1935       // TODO: We can handle/merge duplicate request
1936       ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch();
1937       procId = this.procedureExecutor.submitProcedure(
1938         new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch));
1939       latch.await();
1940     } else {
1941       this.service.submit(new DeleteTableHandler(tableName, this, this).prepare());
1942     }
1943 
1944     if (cpHost != null) {
1945       cpHost.postDeleteTable(tableName);
1946     }
1947 
1948     return procId;
1949   }
1950 
1951   @Override
1952   public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
1953     checkInitialized();
1954     if (cpHost != null) {
1955       cpHost.preTruncateTable(tableName);
1956     }
1957     LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
1958 
1959     if (isMasterProcedureExecutorEnabled()) {
1960       long procId = this.procedureExecutor.submitProcedure(
1961         new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName, preserveSplits));
1962       ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
1963     } else {
1964       TruncateTableHandler handler =
1965           new TruncateTableHandler(tableName, this, this, preserveSplits);
1966       handler.prepare();
1967       handler.process();
1968     }
1969 
1970     if (cpHost != null) {
1971       cpHost.postTruncateTable(tableName);
1972     }
1973   }
1974 
1975   @Override
1976   public void addColumn(final TableName tableName, final HColumnDescriptor columnDescriptor)
1977       throws IOException {
1978     checkInitialized();
1979     checkCompression(columnDescriptor);
1980     checkEncryption(conf, columnDescriptor);
1981     if (cpHost != null) {
1982       if (cpHost.preAddColumn(tableName, columnDescriptor)) {
1983         return;
1984       }
1985     }
1986     LOG.info(getClientIdAuditPrefix() + " add " + columnDescriptor);
1987 
1988     if (isMasterProcedureExecutorEnabled()) {
1989       // Execute the operation synchronously - wait for the operation to complete before continuing.
1990       long procId =
1991         this.procedureExecutor.submitProcedure(new AddColumnFamilyProcedure(procedureExecutor
1992             .getEnvironment(), tableName, columnDescriptor));
1993       ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
1994     } else {
1995       new TableAddFamilyHandler(tableName, columnDescriptor, this, this).prepare().process();
1996     }
1997     if (cpHost != null) {
1998       cpHost.postAddColumn(tableName, columnDescriptor);
1999     }
2000   }
2001 
2002   @Override
2003   public void modifyColumn(TableName tableName, HColumnDescriptor descriptor)
2004       throws IOException {
2005     checkInitialized();
2006     checkCompression(descriptor);
2007     checkEncryption(conf, descriptor);
2008     if (cpHost != null) {
2009       if (cpHost.preModifyColumn(tableName, descriptor)) {
2010         return;
2011       }
2012     }
2013     LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
2014 
2015     if (isMasterProcedureExecutorEnabled()) {
2016       // Execute the operation synchronously - wait for the operation to complete before continuing.
2017       long procId =
2018         this.procedureExecutor.submitProcedure(new ModifyColumnFamilyProcedure(procedureExecutor
2019             .getEnvironment(), tableName, descriptor));
2020       ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
2021     } else {
2022       new TableModifyFamilyHandler(tableName, descriptor, this, this).prepare().process();
2023     }
2024 
2025     if (cpHost != null) {
2026       cpHost.postModifyColumn(tableName, descriptor);
2027     }
2028   }
2029 
2030   @Override
2031   public void deleteColumn(final TableName tableName, final byte[] columnName)
2032       throws IOException {
2033     checkInitialized();
2034     if (cpHost != null) {
2035       if (cpHost.preDeleteColumn(tableName, columnName)) {
2036         return;
2037       }
2038     }
2039     LOG.info(getClientIdAuditPrefix() + " delete " + Bytes.toString(columnName));
2040 
2041     if (isMasterProcedureExecutorEnabled()) {
2042       // Execute the operation synchronously - wait for the operation to complete before continuing.
2043       long procId =
2044         this.procedureExecutor.submitProcedure(new DeleteColumnFamilyProcedure(procedureExecutor
2045             .getEnvironment(), tableName, columnName));
2046       ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
2047     } else {
2048       new TableDeleteFamilyHandler(tableName, columnName, this, this).prepare().process();
2049     }
2050 
2051     if (cpHost != null) {
2052       cpHost.postDeleteColumn(tableName, columnName);
2053     }
2054   }
2055 
2056   @Override
2057   public long enableTable(final TableName tableName) throws IOException {
2058     checkInitialized();
2059     if (cpHost != null) {
2060       cpHost.preEnableTable(tableName);
2061     }
2062     LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
2063 
2064     long procId = -1;
2065     if (isMasterProcedureExecutorEnabled()) {
2066       // Execute the operation asynchronously - client will check the progress of the operation
2067       final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
2068       procId =
2069           this.procedureExecutor.submitProcedure(new EnableTableProcedure(procedureExecutor
2070             .getEnvironment(), tableName, false, prepareLatch));
2071       // Before returning to client, we want to make sure that the table is prepared to be
2072       // enabled (the table is locked and the table state is set).
2073       //
2074       // Note: if the procedure throws exception, we will catch it and rethrow.
2075       prepareLatch.await();
2076     } else {
2077       this.service.submit(new EnableTableHandler(this, tableName,
2078         assignmentManager, tableLockManager, false).prepare());
2079     }
2080 
2081     if (cpHost != null) {
2082       cpHost.postEnableTable(tableName);
2083     }
2084 
2085     return procId;
2086   }
2087 
2088   @Override
2089   public long disableTable(final TableName tableName) throws IOException {
2090     checkInitialized();
2091     if (cpHost != null) {
2092       cpHost.preDisableTable(tableName);
2093     }
2094     LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
2095 
2096     long procId = -1;
2097     if (isMasterProcedureExecutorEnabled()) {
2098       // Execute the operation asynchronously - client will check the progress of the operation
2099       final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
2100       procId =
2101           this.procedureExecutor.submitProcedure(new DisableTableProcedure(procedureExecutor
2102             .getEnvironment(), tableName, false, prepareLatch));
2103       // Before returning to client, we want to make sure that the table is prepared to be
2104       // enabled (the table is locked and the table state is set).
2105       //
2106       // Note: if the procedure throws exception, we will catch it and rethrow.
2107       prepareLatch.await();
2108     } else {
2109       this.service.submit(new DisableTableHandler(this, tableName,
2110         assignmentManager, tableLockManager, false).prepare());
2111     }
2112 
2113     if (cpHost != null) {
2114       cpHost.postDisableTable(tableName);
2115     }
2116 
2117     return procId;
2118   }
2119 
2120   /**
2121    * Return the region and current deployment for the region containing
2122    * the given row. If the region cannot be found, returns null. If it
2123    * is found, but not currently deployed, the second element of the pair
2124    * may be null.
2125    */
2126   @VisibleForTesting // Used by TestMaster.
2127   Pair<HRegionInfo, ServerName> getTableRegionForRow(
2128       final TableName tableName, final byte [] rowKey)
2129   throws IOException {
2130     final AtomicReference<Pair<HRegionInfo, ServerName>> result =
2131       new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
2132 
2133     MetaScannerVisitor visitor =
2134       new MetaScannerVisitorBase() {
2135         @Override
2136         public boolean processRow(Result data) throws IOException {
2137           if (data == null || data.size() <= 0) {
2138             return true;
2139           }
2140           Pair<HRegionInfo, ServerName> pair = HRegionInfo.getHRegionInfoAndServerName(data);
2141           if (pair == null) {
2142             return false;
2143           }
2144           if (!pair.getFirst().getTable().equals(tableName)) {
2145             return false;
2146           }
2147           result.set(pair);
2148           return true;
2149         }
2150     };
2151 
2152     MetaScanner.metaScan(clusterConnection, visitor, tableName, rowKey, 1);
2153     return result.get();
2154   }
2155 
2156   @Override
2157   public void modifyTable(final TableName tableName, final HTableDescriptor descriptor)
2158       throws IOException {
2159     checkInitialized();
2160     sanityCheckTableDescriptor(descriptor);
2161     if (cpHost != null) {
2162       cpHost.preModifyTable(tableName, descriptor);
2163     }
2164 
2165     LOG.info(getClientIdAuditPrefix() + " modify " + tableName);
2166 
2167     if (isMasterProcedureExecutorEnabled()) {
2168       // Execute the operation synchronously - wait for the operation completes before continuing.
2169       long procId = this.procedureExecutor.submitProcedure(
2170         new ModifyTableProcedure(procedureExecutor.getEnvironment(), descriptor));
2171 
2172       ProcedureSyncWait.waitForProcedureToComplete(procedureExecutor, procId);
2173     } else {
2174       new ModifyTableHandler(tableName, descriptor, this, this).prepare().process();
2175     }
2176 
2177     if (cpHost != null) {
2178       cpHost.postModifyTable(tableName, descriptor);
2179     }
2180   }
2181 
2182   @Override
2183   public void checkTableModifiable(final TableName tableName)
2184       throws IOException, TableNotFoundException, TableNotDisabledException {
2185     if (isCatalogTable(tableName)) {
2186       throw new IOException("Can't modify catalog tables");
2187     }
2188     if (!MetaTableAccessor.tableExists(getConnection(), tableName)) {
2189       throw new TableNotFoundException(tableName);
2190     }
2191     if (!getAssignmentManager().getTableStateManager().
2192         isTableState(tableName, ZooKeeperProtos.Table.State.DISABLED)) {
2193       throw new TableNotDisabledException(tableName);
2194     }
2195   }
2196 
2197   /**
2198    * @return cluster status
2199    */
2200   public ClusterStatus getClusterStatus() throws InterruptedIOException {
2201     // Build Set of backup masters from ZK nodes
2202     List<String> backupMasterStrings;
2203     try {
2204       backupMasterStrings = ZKUtil.listChildrenNoWatch(this.zooKeeper,
2205         this.zooKeeper.backupMasterAddressesZNode);
2206     } catch (KeeperException e) {
2207       LOG.warn(this.zooKeeper.prefix("Unable to list backup servers"), e);
2208       backupMasterStrings = null;
2209     }
2210 
2211     List<ServerName> backupMasters = null;
2212     if (backupMasterStrings != null && !backupMasterStrings.isEmpty()) {
2213       backupMasters = new ArrayList<ServerName>(backupMasterStrings.size());
2214       for (String s: backupMasterStrings) {
2215         try {
2216           byte [] bytes;
2217           try {
2218             bytes = ZKUtil.getData(this.zooKeeper, ZKUtil.joinZNode(
2219                 this.zooKeeper.backupMasterAddressesZNode, s));
2220           } catch (InterruptedException e) {
2221             throw new InterruptedIOException();
2222           }
2223           if (bytes != null) {
2224             ServerName sn;
2225             try {
2226               sn = ServerName.parseFrom(bytes);
2227             } catch (DeserializationException e) {
2228               LOG.warn("Failed parse, skipping registering backup server", e);
2229               continue;
2230             }
2231             backupMasters.add(sn);
2232           }
2233         } catch (KeeperException e) {
2234           LOG.warn(this.zooKeeper.prefix("Unable to get information about " +
2235                    "backup servers"), e);
2236         }
2237       }
2238       Collections.sort(backupMasters, new Comparator<ServerName>() {
2239         @Override
2240         public int compare(ServerName s1, ServerName s2) {
2241           return s1.getServerName().compareTo(s2.getServerName());
2242         }});
2243     }
2244 
2245     String clusterId = fileSystemManager != null ?
2246       fileSystemManager.getClusterId().toString() : null;
2247     Map<String, RegionState> regionsInTransition = assignmentManager != null ?
2248       assignmentManager.getRegionStates().getRegionsInTransition() : null;
2249     String[] coprocessors = cpHost != null ? getMasterCoprocessors() : null;
2250     boolean balancerOn = loadBalancerTracker != null ?
2251       loadBalancerTracker.isBalancerOn() : false;
2252     Map<ServerName, ServerLoad> onlineServers = null;
2253     Set<ServerName> deadServers = null;
2254     if (serverManager != null) {
2255       deadServers = serverManager.getDeadServers().copyServerNames();
2256       onlineServers = serverManager.getOnlineServers();
2257     }
2258     return new ClusterStatus(VersionInfo.getVersion(), clusterId,
2259       onlineServers, deadServers, serverName, backupMasters,
2260       regionsInTransition, coprocessors, balancerOn);
2261   }
2262 
2263   /**
2264    * The set of loaded coprocessors is stored in a static set. Since it's
2265    * statically allocated, it does not require that HMaster's cpHost be
2266    * initialized prior to accessing it.
2267    * @return a String representation of the set of names of the loaded
2268    * coprocessors.
2269    */
2270   public static String getLoadedCoprocessors() {
2271     return CoprocessorHost.getLoadedCoprocessors().toString();
2272   }
2273 
2274   /**
2275    * @return timestamp in millis when HMaster was started.
2276    */
2277   public long getMasterStartTime() {
2278     return startcode;
2279   }
2280 
2281   /**
2282    * @return timestamp in millis when HMaster became the active master.
2283    */
2284   public long getMasterActiveTime() {
2285     return masterActiveTime;
2286   }
2287 
2288   public int getNumWALFiles() {
2289     return procedureStore != null ? procedureStore.getActiveLogs().size() : 0;
2290   }
2291 
2292   public WALProcedureStore getWalProcedureStore() {
2293     return procedureStore;
2294   }
2295 
2296   public int getRegionServerInfoPort(final ServerName sn) {
2297     RegionServerInfo info = this.regionServerTracker.getRegionServerInfo(sn);
2298     if (info == null || info.getInfoPort() == 0) {
2299       return conf.getInt(HConstants.REGIONSERVER_INFO_PORT,
2300         HConstants.DEFAULT_REGIONSERVER_INFOPORT);
2301     }
2302     return info.getInfoPort();
2303   }
2304 
2305   /**
2306    * @return array of coprocessor SimpleNames.
2307    */
2308   public String[] getMasterCoprocessors() {
2309     Set<String> masterCoprocessors = getMasterCoprocessorHost().getCoprocessors();
2310     return masterCoprocessors.toArray(new String[masterCoprocessors.size()]);
2311   }
2312 
2313   @Override
2314   public void abort(final String msg, final Throwable t) {
2315     if (isAborted() || isStopped()) {
2316       return;
2317     }
2318     if (cpHost != null) {
2319       // HBASE-4014: dump a list of loaded coprocessors.
2320       LOG.fatal("Master server abort: loaded coprocessors are: " +
2321           getLoadedCoprocessors());
2322     }
2323     if (t != null) LOG.fatal(msg, t);
2324     stop(msg);
2325   }
2326 
2327   @Override
2328   public ZooKeeperWatcher getZooKeeper() {
2329     return zooKeeper;
2330   }
2331 
2332   @Override
2333   public MasterCoprocessorHost getMasterCoprocessorHost() {
2334     return cpHost;
2335   }
2336 
2337   @Override
2338   public MasterQuotaManager getMasterQuotaManager() {
2339     return quotaManager;
2340   }
2341 
2342   @Override
2343   public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
2344     return procedureExecutor;
2345   }
2346 
2347   @Override
2348   public ServerName getServerName() {
2349     return this.serverName;
2350   }
2351 
2352   @Override
2353   public AssignmentManager getAssignmentManager() {
2354     return this.assignmentManager;
2355   }
2356 
2357   public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
2358     return rsFatals;
2359   }
2360 
2361   public void shutdown() {
2362     if (cpHost != null) {
2363       try {
2364         cpHost.preShutdown();
2365       } catch (IOException ioe) {
2366         LOG.error("Error call master coprocessor preShutdown()", ioe);
2367       }
2368     }
2369 
2370     if (this.serverManager != null) {
2371       this.serverManager.shutdownCluster();
2372     }
2373     if (this.clusterStatusTracker != null){
2374       try {
2375         this.clusterStatusTracker.setClusterDown();
2376       } catch (KeeperException e) {
2377         LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
2378       }
2379     }
2380   }
2381 
2382   public void stopMaster() {
2383     if (cpHost != null) {
2384       try {
2385         cpHost.preStopMaster();
2386       } catch (IOException ioe) {
2387         LOG.error("Error call master coprocessor preStopMaster()", ioe);
2388       }
2389     }
2390     stop("Stopped by " + Thread.currentThread().getName());
2391   }
2392 
2393   void checkServiceStarted() throws ServerNotRunningYetException {
2394     if (!serviceStarted) {
2395       throw new ServerNotRunningYetException("Server is not running yet");
2396     }
2397   }
2398 
2399   void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException {
2400     checkServiceStarted();
2401     if (!this.initialized) {
2402       throw new PleaseHoldException("Master is initializing");
2403     }
2404   }
2405 
2406   void checkNamespaceManagerReady() throws IOException {
2407     checkInitialized();
2408     if (tableNamespaceManager == null ||
2409         !tableNamespaceManager.isTableAvailableAndInitialized()) {
2410       throw new IOException("Table Namespace Manager not ready yet, try again later");
2411     }
2412   }
2413   /**
2414    * Report whether this master is currently the active master or not.
2415    * If not active master, we are parked on ZK waiting to become active.
2416    *
2417    * This method is used for testing.
2418    *
2419    * @return true if active master, false if not.
2420    */
2421   public boolean isActiveMaster() {
2422     return isActiveMaster;
2423   }
2424 
2425   /**
2426    * Report whether this master has completed with its initialization and is
2427    * ready.  If ready, the master is also the active master.  A standby master
2428    * is never ready.
2429    *
2430    * This method is used for testing.
2431    *
2432    * @return true if master is ready to go, false if not.
2433    */
2434   @Override
2435   public boolean isInitialized() {
2436     return initialized;
2437   }
2438 
2439   /**
2440    * ServerShutdownHandlerEnabled is set false before completing
2441    * assignMeta to prevent processing of ServerShutdownHandler.
2442    * @return true if assignMeta has completed;
2443    */
2444   @Override
2445   public boolean isServerShutdownHandlerEnabled() {
2446     return this.serverShutdownHandlerEnabled;
2447   }
2448 
2449   /**
2450    * Report whether this master has started initialization and is about to do meta region assignment
2451    * @return true if master is in initialization & about to assign hbase:meta regions
2452    */
2453   public boolean isInitializationStartsMetaRegionAssignment() {
2454     return this.initializationBeforeMetaAssignment;
2455   }
2456 
2457   public void assignRegion(HRegionInfo hri) {
2458     assignmentManager.assign(hri, true);
2459   }
2460 
2461   /**
2462    * Compute the average load across all region servers.
2463    * Currently, this uses a very naive computation - just uses the number of
2464    * regions being served, ignoring stats about number of requests.
2465    * @return the average load
2466    */
2467   public double getAverageLoad() {
2468     if (this.assignmentManager == null) {
2469       return 0;
2470     }
2471 
2472     RegionStates regionStates = this.assignmentManager.getRegionStates();
2473     if (regionStates == null) {
2474       return 0;
2475     }
2476     return regionStates.getAverageLoad();
2477   }
2478 
2479   @Override
2480   public boolean registerService(Service instance) {
2481     /*
2482      * No stacking of instances is allowed for a single service name
2483      */
2484     Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
2485     if (coprocessorServiceHandlers.containsKey(serviceDesc.getFullName())) {
2486       LOG.error("Coprocessor service "+serviceDesc.getFullName()+
2487           " already registered, rejecting request from "+instance
2488       );
2489       return false;
2490     }
2491 
2492     coprocessorServiceHandlers.put(serviceDesc.getFullName(), instance);
2493     if (LOG.isDebugEnabled()) {
2494       LOG.debug("Registered master coprocessor service: service="+serviceDesc.getFullName());
2495     }
2496     return true;
2497   }
2498 
2499   /**
2500    * Utility for constructing an instance of the passed HMaster class.
2501    * @param masterClass
2502    * @param conf
2503    * @return HMaster instance.
2504    */
2505   public static HMaster constructMaster(Class<? extends HMaster> masterClass,
2506       final Configuration conf, final CoordinatedStateManager cp)  {
2507     try {
2508       Constructor<? extends HMaster> c =
2509         masterClass.getConstructor(Configuration.class, CoordinatedStateManager.class);
2510       return c.newInstance(conf, cp);
2511     } catch (InvocationTargetException ite) {
2512       Throwable target = ite.getTargetException() != null?
2513         ite.getTargetException(): ite;
2514       if (target.getCause() != null) target = target.getCause();
2515       throw new RuntimeException("Failed construction of Master: " +
2516         masterClass.toString(), target);
2517     } catch (Exception e) {
2518       throw new RuntimeException("Failed construction of Master: " +
2519         masterClass.toString() + ((e.getCause() != null)?
2520           e.getCause().getMessage(): ""), e);
2521     }
2522   }
2523 
2524   /**
2525    * @see org.apache.hadoop.hbase.master.HMasterCommandLine
2526    */
2527   public static void main(String [] args) {
2528     VersionInfo.logVersion();
2529     new HMasterCommandLine(HMaster.class).doMain(args);
2530   }
2531 
2532   public HFileCleaner getHFileCleaner() {
2533     return this.hfileCleaner;
2534   }
2535 
2536   /**
2537    * @return the underlying snapshot manager
2538    */
2539   public SnapshotManager getSnapshotManager() {
2540     return this.snapshotManager;
2541   }
2542 
2543   /**
2544    * @return the underlying MasterProcedureManagerHost
2545    */
2546   public MasterProcedureManagerHost getMasterProcedureManagerHost() {
2547     return mpmHost;
2548   }
2549 
2550   @Override
2551   public void createNamespace(NamespaceDescriptor descriptor) throws IOException {
2552     TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
2553     checkNamespaceManagerReady();
2554     if (cpHost != null) {
2555       if (cpHost.preCreateNamespace(descriptor)) {
2556         return;
2557       }
2558     }
2559     LOG.info(getClientIdAuditPrefix() + " creating " + descriptor);
2560     tableNamespaceManager.create(descriptor);
2561     if (cpHost != null) {
2562       cpHost.postCreateNamespace(descriptor);
2563     }
2564   }
2565 
2566   @Override
2567   public void modifyNamespace(NamespaceDescriptor descriptor) throws IOException {
2568     TableName.isLegalNamespaceName(Bytes.toBytes(descriptor.getName()));
2569     checkNamespaceManagerReady();
2570     if (cpHost != null) {
2571       if (cpHost.preModifyNamespace(descriptor)) {
2572         return;
2573       }
2574     }
2575     LOG.info(getClientIdAuditPrefix() + " modify " + descriptor);
2576     tableNamespaceManager.update(descriptor);
2577     if (cpHost != null) {
2578       cpHost.postModifyNamespace(descriptor);
2579     }
2580   }
2581 
2582   @Override
2583   public void deleteNamespace(String name) throws IOException {
2584     checkNamespaceManagerReady();
2585     if (cpHost != null) {
2586       if (cpHost.preDeleteNamespace(name)) {
2587         return;
2588       }
2589     }
2590     LOG.info(getClientIdAuditPrefix() + " delete " + name);
2591     tableNamespaceManager.remove(name);
2592     if (cpHost != null) {
2593       cpHost.postDeleteNamespace(name);
2594     }
2595   }
2596 
2597   /**
2598    * Ensure that the specified namespace exists, otherwise throws a NamespaceNotFoundException
2599    *
2600    * @param name the namespace to check
2601    * @throws IOException if the namespace manager is not ready yet.
2602    * @throws NamespaceNotFoundException if the namespace does not exists
2603    */
2604   private void ensureNamespaceExists(final String name)
2605       throws IOException, NamespaceNotFoundException {
2606     checkNamespaceManagerReady();
2607     NamespaceDescriptor nsd = tableNamespaceManager.get(name);
2608     if (nsd == null) {
2609       throw new NamespaceNotFoundException(name);
2610     }
2611   }
2612 
2613   @Override
2614   public NamespaceDescriptor getNamespaceDescriptor(String name) throws IOException {
2615     checkNamespaceManagerReady();
2616 
2617     if (cpHost != null) {
2618       cpHost.preGetNamespaceDescriptor(name);
2619     }
2620 
2621     NamespaceDescriptor nsd = tableNamespaceManager.get(name);
2622     if (nsd == null) {
2623       throw new NamespaceNotFoundException(name);
2624     }
2625 
2626     if (cpHost != null) {
2627       cpHost.postGetNamespaceDescriptor(nsd);
2628     }
2629 
2630     return nsd;
2631   }
2632 
2633   @Override
2634   public List<NamespaceDescriptor> listNamespaceDescriptors() throws IOException {
2635     checkNamespaceManagerReady();
2636 
2637     final List<NamespaceDescriptor> descriptors = new ArrayList<NamespaceDescriptor>();
2638     boolean bypass = false;
2639     if (cpHost != null) {
2640       bypass = cpHost.preListNamespaceDescriptors(descriptors);
2641     }
2642 
2643     if (!bypass) {
2644       descriptors.addAll(tableNamespaceManager.list());
2645 
2646       if (cpHost != null) {
2647         cpHost.postListNamespaceDescriptors(descriptors);
2648       }
2649     }
2650     return descriptors;
2651   }
2652 
2653   @Override
2654   public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
2655       throws IOException {
2656     if (cpHost != null) {
2657       cpHost.preAbortProcedure(this.procedureExecutor, procId);
2658     }
2659 
2660     final boolean result = this.procedureExecutor.abort(procId, mayInterruptIfRunning);
2661 
2662     if (cpHost != null) {
2663       cpHost.postAbortProcedure();
2664     }
2665 
2666     return result;
2667   }
2668 
2669   @Override
2670   public List<ProcedureInfo> listProcedures() throws IOException {
2671     if (cpHost != null) {
2672       cpHost.preListProcedures();
2673     }
2674 
2675     final List<ProcedureInfo> procInfoList = this.procedureExecutor.listProcedures();
2676 
2677     if (cpHost != null) {
2678       cpHost.postListProcedures(procInfoList);
2679     }
2680 
2681     return procInfoList;
2682   }
2683 
2684   @Override
2685   public List<HTableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
2686     ensureNamespaceExists(name);
2687     return listTableDescriptors(name, null, null, true);
2688   }
2689 
2690   @Override
2691   public List<TableName> listTableNamesByNamespace(String name) throws IOException {
2692     ensureNamespaceExists(name);
2693     return listTableNames(name, null, true);
2694   }
2695 
2696   @Override
2697   public Pair<Long, String> backupTables(final BackupType type,
2698         List<TableName> tableList, final String targetRootDir, final int workers,
2699         final long bandwidth) throws IOException {
2700     long procId;
2701     String backupId = BackupRestoreConstants.BACKUPID_PREFIX + 
2702         EnvironmentEdgeManager.currentTime();
2703     if (type == BackupType.INCREMENTAL) {
2704       Set<TableName> incrTableSet = null;
2705       try (BackupSystemTable table = new BackupSystemTable(getConnection())) {
2706         incrTableSet = table.getIncrementalBackupTableSet(targetRootDir);
2707       }
2708          
2709       if (incrTableSet.isEmpty()) {
2710         LOG.warn("Incremental backup table set contains no table.\n"
2711             + "Use 'backup create full' or 'backup stop' to \n "
2712             + "change the tables covered by incremental backup.");
2713         throw new DoNotRetryIOException("No table covered by incremental backup.");
2714       }
2715 
2716       LOG.info("Incremental backup for the following table set: " + incrTableSet);
2717       tableList = Lists.newArrayList(incrTableSet);
2718     }
2719     if (tableList != null && !tableList.isEmpty()) {
2720       for (TableName table : tableList) {
2721         String targetTableBackupDir =
2722             HBackupFileSystem.getTableBackupDir(targetRootDir, backupId, table);
2723         Path targetTableBackupDirPath = new Path(targetTableBackupDir);
2724         FileSystem outputFs = FileSystem.get(targetTableBackupDirPath.toUri(), conf);
2725         if (outputFs.exists(targetTableBackupDirPath)) {
2726           throw new DoNotRetryIOException("Target backup directory " + targetTableBackupDir
2727             + " exists already.");
2728         }
2729       }
2730       ArrayList<TableName> nonExistingTableList = null;
2731       for (TableName tableName : tableList) {
2732         if (!MetaTableAccessor.tableExists(getConnection(), tableName)) {
2733           if (nonExistingTableList == null) {
2734             nonExistingTableList = new ArrayList<>();
2735           }
2736           nonExistingTableList.add(tableName);
2737         }
2738       }
2739       if (nonExistingTableList != null) {
2740         if (type == BackupType.INCREMENTAL ) {
2741           LOG.warn("Incremental backup table set contains non-exising table: "
2742               + nonExistingTableList);
2743           // Update incremental backup set 
2744           tableList = excludeNonExistingTables(tableList, nonExistingTableList);
2745         } else {
2746           // Throw exception only in full mode - we try to backup non-existing table
2747           throw new DoNotRetryIOException("Non-existing tables found in the table list: "
2748               + nonExistingTableList);
2749         }
2750       }
2751     }
2752     if (type == BackupType.FULL) {
2753       procId = this.procedureExecutor.submitProcedure(
2754         new FullTableBackupProcedure(procedureExecutor.getEnvironment(), backupId,
2755           tableList, targetRootDir, workers, bandwidth));
2756     } else {
2757       procId = this.procedureExecutor.submitProcedure(
2758         new IncrementalTableBackupProcedure(procedureExecutor.getEnvironment(), backupId,
2759           tableList, targetRootDir, workers, bandwidth));
2760     }
2761     return new Pair<>(procId, backupId);
2762   }
2763 
2764   private List<TableName> excludeNonExistingTables(List<TableName> tableList,
2765       List<TableName> nonExistingTableList) {
2766     
2767     for(TableName table: nonExistingTableList) {
2768       tableList.remove(table);
2769     }
2770     return tableList;
2771   }
2772 
2773   /**
2774    * Returns the list of table descriptors that match the specified request
2775    *
2776    * @param namespace the namespace to query, or null if querying for all
2777    * @param regex The regular expression to match against, or null if querying for all
2778    * @param tableNameList the list of table names, or null if querying for all
2779    * @param includeSysTables False to match only against userspace tables
2780    * @return the list of table descriptors
2781    */
2782   public List<HTableDescriptor> listTableDescriptors(final String namespace, final String regex,
2783       final List<TableName> tableNameList, final boolean includeSysTables)
2784       throws IOException {
2785     final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
2786 
2787     boolean bypass = false;
2788     if (cpHost != null) {
2789       bypass = cpHost.preGetTableDescriptors(tableNameList, descriptors);
2790       // method required for AccessController.
2791       bypass |= cpHost.preGetTableDescriptors(tableNameList, descriptors, regex);
2792     }
2793 
2794     if (!bypass) {
2795       if (tableNameList == null || tableNameList.size() == 0) {
2796         // request for all TableDescriptors
2797         Collection<HTableDescriptor> htds;
2798         if (namespace != null && namespace.length() > 0) {
2799           htds = tableDescriptors.getByNamespace(namespace).values();
2800         } else {
2801           htds = tableDescriptors.getAll().values();
2802         }
2803 
2804         for (HTableDescriptor desc: htds) {
2805           if (includeSysTables || !desc.getTableName().isSystemTable()) {
2806             descriptors.add(desc);
2807           }
2808         }
2809       } else {
2810         for (TableName s: tableNameList) {
2811           HTableDescriptor desc = tableDescriptors.get(s);
2812           if (desc != null) {
2813             descriptors.add(desc);
2814           }
2815         }
2816       }
2817 
2818       // Retains only those matched by regular expression.
2819       if (regex != null) {
2820         filterTablesByRegex(descriptors, Pattern.compile(regex));
2821       }
2822 
2823       if (cpHost != null) {
2824         cpHost.postGetTableDescriptors(descriptors);
2825         // method required for AccessController.
2826         cpHost.postGetTableDescriptors(tableNameList, descriptors, regex);
2827       }
2828     }
2829     return descriptors;
2830   }
2831 
2832   /**
2833    * Returns the list of table names that match the specified request
2834    * @param regex The regular expression to match against, or null if querying for all
2835    * @param namespace the namespace to query, or null if querying for all
2836    * @param includeSysTables False to match only against userspace tables
2837    * @return the list of table names
2838    */
2839   public List<TableName> listTableNames(final String namespace, final String regex,
2840       final boolean includeSysTables) throws IOException {
2841     final List<HTableDescriptor> descriptors = new ArrayList<HTableDescriptor>();
2842 
2843     boolean bypass = false;
2844     if (cpHost != null) {
2845       bypass = cpHost.preGetTableNames(descriptors, regex);
2846     }
2847 
2848     if (!bypass) {
2849       // get all descriptors
2850       Collection<HTableDescriptor> htds;
2851       if (namespace != null && namespace.length() > 0) {
2852         htds = tableDescriptors.getByNamespace(namespace).values();
2853       } else {
2854         htds = tableDescriptors.getAll().values();
2855       }
2856 
2857       for (HTableDescriptor htd: htds) {
2858         if (includeSysTables || !htd.getTableName().isSystemTable()) {
2859           descriptors.add(htd);
2860         }
2861       }
2862 
2863       // Retains only those matched by regular expression.
2864       if (regex != null) {
2865         filterTablesByRegex(descriptors, Pattern.compile(regex));
2866       }
2867 
2868       if (cpHost != null) {
2869         cpHost.postGetTableNames(descriptors, regex);
2870       }
2871     }
2872 
2873     List<TableName> result = new ArrayList<TableName>(descriptors.size());
2874     for (HTableDescriptor htd: descriptors) {
2875       result.add(htd.getTableName());
2876     }
2877     return result;
2878   }
2879 
2880 
2881   /**
2882    * Removes the table descriptors that don't match the pattern.
2883    * @param descriptors list of table descriptors to filter
2884    * @param pattern the regex to use
2885    */
2886   private static void filterTablesByRegex(final Collection<HTableDescriptor> descriptors,
2887       final Pattern pattern) {
2888     final String defaultNS = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
2889     Iterator<HTableDescriptor> itr = descriptors.iterator();
2890     while (itr.hasNext()) {
2891       HTableDescriptor htd = itr.next();
2892       String tableName = htd.getTableName().getNameAsString();
2893       boolean matched = pattern.matcher(tableName).matches();
2894       if (!matched && htd.getTableName().getNamespaceAsString().equals(defaultNS)) {
2895         matched = pattern.matcher(defaultNS + TableName.NAMESPACE_DELIM + tableName).matches();
2896       }
2897       if (!matched) {
2898         itr.remove();
2899       }
2900     }
2901   }
2902 
2903   @Override
2904   public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
2905     return getClusterStatus().getLastMajorCompactionTsForTable(table);
2906   }
2907 
2908   @Override
2909   public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
2910     return getClusterStatus().getLastMajorCompactionTsForRegion(regionName);
2911   }
2912 
2913   /**
2914    * Gets the mob file compaction state for a specific table.
2915    * Whether all the mob files are selected is known during the compaction execution, but
2916    * the statistic is done just before compaction starts, it is hard to know the compaction
2917    * type at that time, so the rough statistics are chosen for the mob file compaction. Only two
2918    * compaction states are available, CompactionState.MAJOR_AND_MINOR and CompactionState.NONE.
2919    * @param tableName The current table name.
2920    * @return If a given table is in mob file compaction now.
2921    */
2922   public CompactionState getMobCompactionState(TableName tableName) {
2923     AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
2924     if (compactionsCount != null && compactionsCount.get() != 0) {
2925       return CompactionState.MAJOR_AND_MINOR;
2926     }
2927     return CompactionState.NONE;
2928   }
2929 
2930   public void reportMobCompactionStart(TableName tableName) throws IOException {
2931     IdLock.Entry lockEntry = null;
2932     try {
2933       lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
2934       AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
2935       if (compactionsCount == null) {
2936         compactionsCount = new AtomicInteger(0);
2937         mobCompactionStates.put(tableName, compactionsCount);
2938       }
2939       compactionsCount.incrementAndGet();
2940     } finally {
2941       if (lockEntry != null) {
2942         mobCompactionLock.releaseLockEntry(lockEntry);
2943       }
2944     }
2945   }
2946 
2947   public void reportMobCompactionEnd(TableName tableName) throws IOException {
2948     IdLock.Entry lockEntry = null;
2949     try {
2950       lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
2951       AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
2952       if (compactionsCount != null) {
2953         int count = compactionsCount.decrementAndGet();
2954         // remove the entry if the count is 0.
2955         if (count == 0) {
2956           mobCompactionStates.remove(tableName);
2957         }
2958       }
2959     } finally {
2960       if (lockEntry != null) {
2961         mobCompactionLock.releaseLockEntry(lockEntry);
2962       }
2963     }
2964   }
2965 
2966   /**
2967    * Requests mob compaction.
2968    * @param tableName The table the compact.
2969    * @param columns The compacted columns.
2970    * @param allFiles Whether add all mob files into the compaction.
2971    */
2972   public void requestMobCompaction(TableName tableName,
2973     List<HColumnDescriptor> columns, boolean allFiles) throws IOException {
2974     mobCompactThread.requestMobCompaction(conf, fs, tableName, columns,
2975       tableLockManager, allFiles);
2976   }
2977 
2978   /**
2979    * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized,
2980    * false is returned.
2981    *
2982    * @return The state of the load balancer, or false if the load balancer isn't defined.
2983    */
2984   public boolean isBalancerOn() {
2985     if (null == loadBalancerTracker) return false;
2986     return loadBalancerTracker.isBalancerOn();
2987   }
2988 
2989   /**
2990    * Queries the state of the {@link RegionNormalizerTracker}. If it's not initialized,
2991    * false is returned.
2992    */
2993    public boolean isNormalizerOn() {
2994     if (null == regionNormalizerTracker) {
2995       return false;
2996     }
2997     return regionNormalizerTracker.isNormalizerOn();
2998   }
2999 
3000 
3001   /**
3002    * Queries the state of the {@link SplitOrMergeTracker}. If it is not initialized,
3003    * false is returned. If switchType is illegal, false will return.
3004    * @param switchType see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType}
3005    * @return The state of the switch
3006    */
3007   public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType) {
3008     if (null == splitOrMergeTracker) {
3009       return false;
3010     }
3011     return splitOrMergeTracker.isSplitOrMergeEnabled(switchType);
3012   }
3013 
3014   /**
3015    * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.
3016    *
3017    * @return The name of the {@link LoadBalancer} in use.
3018    */
3019   public String getLoadBalancerClassName() {
3020     return conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, LoadBalancerFactory
3021         .getDefaultLoadBalancerClass().getName());
3022   }
3023 
3024   /**
3025    * @return RegionNormalizerTracker instance
3026    */
3027   public RegionNormalizerTracker getRegionNormalizerTracker() {
3028     return regionNormalizerTracker;
3029   }
3030 
3031   public SplitOrMergeTracker getSplitOrMergeTracker() {
3032     return splitOrMergeTracker;
3033   }
3034 
3035   @Override
3036   public LoadBalancer getLoadBalancer() {
3037     return balancer;
3038   }
3039 
3040   @Override
3041   public TableStateManager getTableStateManager() {
3042     return assignmentManager.getTableStateManager();
3043   }
3044 }