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