View Javadoc

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