001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.master;
019
020import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK;
021import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS;
022import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_ZK;
023import static org.apache.hadoop.hbase.util.DNS.MASTER_HOSTNAME_KEY;
024
025import com.google.errorprone.annotations.RestrictedApi;
026import java.io.IOException;
027import java.io.InterruptedIOException;
028import java.lang.reflect.Constructor;
029import java.lang.reflect.InvocationTargetException;
030import java.net.InetAddress;
031import java.net.InetSocketAddress;
032import java.net.UnknownHostException;
033import java.util.ArrayList;
034import java.util.Arrays;
035import java.util.Collection;
036import java.util.Collections;
037import java.util.Comparator;
038import java.util.EnumSet;
039import java.util.HashMap;
040import java.util.Iterator;
041import java.util.LinkedList;
042import java.util.List;
043import java.util.Map;
044import java.util.Objects;
045import java.util.Optional;
046import java.util.Set;
047import java.util.concurrent.ExecutionException;
048import java.util.concurrent.Future;
049import java.util.concurrent.TimeUnit;
050import java.util.concurrent.TimeoutException;
051import java.util.concurrent.atomic.AtomicInteger;
052import java.util.regex.Pattern;
053import java.util.stream.Collectors;
054import javax.servlet.http.HttpServlet;
055import org.apache.commons.lang3.StringUtils;
056import org.apache.hadoop.conf.Configuration;
057import org.apache.hadoop.fs.FSDataOutputStream;
058import org.apache.hadoop.fs.Path;
059import org.apache.hadoop.hbase.CatalogFamilyFormat;
060import org.apache.hadoop.hbase.Cell;
061import org.apache.hadoop.hbase.CellBuilderFactory;
062import org.apache.hadoop.hbase.CellBuilderType;
063import org.apache.hadoop.hbase.ClusterId;
064import org.apache.hadoop.hbase.ClusterMetrics;
065import org.apache.hadoop.hbase.ClusterMetrics.Option;
066import org.apache.hadoop.hbase.ClusterMetricsBuilder;
067import org.apache.hadoop.hbase.DoNotRetryIOException;
068import org.apache.hadoop.hbase.HBaseIOException;
069import org.apache.hadoop.hbase.HBaseInterfaceAudience;
070import org.apache.hadoop.hbase.HBaseServerBase;
071import org.apache.hadoop.hbase.HConstants;
072import org.apache.hadoop.hbase.HRegionLocation;
073import org.apache.hadoop.hbase.InvalidFamilyOperationException;
074import org.apache.hadoop.hbase.MasterNotRunningException;
075import org.apache.hadoop.hbase.MetaTableAccessor;
076import org.apache.hadoop.hbase.NamespaceDescriptor;
077import org.apache.hadoop.hbase.PleaseHoldException;
078import org.apache.hadoop.hbase.PleaseRestartMasterException;
079import org.apache.hadoop.hbase.RegionMetrics;
080import org.apache.hadoop.hbase.ReplicationPeerNotFoundException;
081import org.apache.hadoop.hbase.ServerMetrics;
082import org.apache.hadoop.hbase.ServerName;
083import org.apache.hadoop.hbase.ServerTask;
084import org.apache.hadoop.hbase.ServerTaskBuilder;
085import org.apache.hadoop.hbase.TableName;
086import org.apache.hadoop.hbase.TableNotDisabledException;
087import org.apache.hadoop.hbase.TableNotFoundException;
088import org.apache.hadoop.hbase.UnknownRegionException;
089import org.apache.hadoop.hbase.client.BalanceRequest;
090import org.apache.hadoop.hbase.client.BalanceResponse;
091import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
092import org.apache.hadoop.hbase.client.CompactionState;
093import org.apache.hadoop.hbase.client.MasterSwitchType;
094import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
095import org.apache.hadoop.hbase.client.Put;
096import org.apache.hadoop.hbase.client.RegionInfo;
097import org.apache.hadoop.hbase.client.RegionInfoBuilder;
098import org.apache.hadoop.hbase.client.RegionStatesCount;
099import org.apache.hadoop.hbase.client.ResultScanner;
100import org.apache.hadoop.hbase.client.Scan;
101import org.apache.hadoop.hbase.client.TableDescriptor;
102import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
103import org.apache.hadoop.hbase.client.TableState;
104import org.apache.hadoop.hbase.conf.ConfigurationManager;
105import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
106import org.apache.hadoop.hbase.exceptions.MasterStoppedException;
107import org.apache.hadoop.hbase.executor.ExecutorType;
108import org.apache.hadoop.hbase.favored.FavoredNodesManager;
109import org.apache.hadoop.hbase.http.HttpServer;
110import org.apache.hadoop.hbase.http.InfoServer;
111import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
112import org.apache.hadoop.hbase.ipc.RpcServer;
113import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
114import org.apache.hadoop.hbase.log.HBaseMarkers;
115import org.apache.hadoop.hbase.master.MasterRpcServices.BalanceSwitchMode;
116import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
117import org.apache.hadoop.hbase.master.assignment.MergeTableRegionsProcedure;
118import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
119import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
120import org.apache.hadoop.hbase.master.assignment.RegionStates;
121import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
122import org.apache.hadoop.hbase.master.balancer.BalancerChore;
123import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
124import org.apache.hadoop.hbase.master.balancer.ClusterStatusChore;
125import org.apache.hadoop.hbase.master.balancer.LoadBalancerFactory;
126import org.apache.hadoop.hbase.master.balancer.MaintenanceLoadBalancer;
127import org.apache.hadoop.hbase.master.cleaner.DirScanPool;
128import org.apache.hadoop.hbase.master.cleaner.HFileCleaner;
129import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
130import org.apache.hadoop.hbase.master.cleaner.ReplicationBarrierCleaner;
131import org.apache.hadoop.hbase.master.cleaner.SnapshotCleanerChore;
132import org.apache.hadoop.hbase.master.http.MasterDumpServlet;
133import org.apache.hadoop.hbase.master.http.MasterRedirectServlet;
134import org.apache.hadoop.hbase.master.http.MasterStatusServlet;
135import org.apache.hadoop.hbase.master.http.api_v1.ResourceConfigFactory;
136import org.apache.hadoop.hbase.master.janitor.CatalogJanitor;
137import org.apache.hadoop.hbase.master.locking.LockManager;
138import org.apache.hadoop.hbase.master.migrate.RollingUpgradeChore;
139import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerFactory;
140import org.apache.hadoop.hbase.master.normalizer.RegionNormalizerManager;
141import org.apache.hadoop.hbase.master.procedure.CreateTableProcedure;
142import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
143import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
144import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
145import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
146import org.apache.hadoop.hbase.master.procedure.InitMetaProcedure;
147import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
148import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
149import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
150import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
151import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil.NonceProcedureRunnable;
152import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
153import org.apache.hadoop.hbase.master.procedure.ProcedurePrepareLatch;
154import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
155import org.apache.hadoop.hbase.master.procedure.ReopenTableRegionsProcedure;
156import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
157import org.apache.hadoop.hbase.master.procedure.TruncateTableProcedure;
158import org.apache.hadoop.hbase.master.region.MasterRegion;
159import org.apache.hadoop.hbase.master.region.MasterRegionFactory;
160import org.apache.hadoop.hbase.master.replication.AbstractPeerProcedure;
161import org.apache.hadoop.hbase.master.replication.AddPeerProcedure;
162import org.apache.hadoop.hbase.master.replication.DisablePeerProcedure;
163import org.apache.hadoop.hbase.master.replication.EnablePeerProcedure;
164import org.apache.hadoop.hbase.master.replication.RemovePeerProcedure;
165import org.apache.hadoop.hbase.master.replication.ReplicationPeerManager;
166import org.apache.hadoop.hbase.master.replication.SyncReplicationReplayWALManager;
167import org.apache.hadoop.hbase.master.replication.TransitPeerSyncReplicationStateProcedure;
168import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure;
169import org.apache.hadoop.hbase.master.slowlog.SlowLogMasterService;
170import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
171import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer;
172import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer;
173import org.apache.hadoop.hbase.mob.MobFileCleanerChore;
174import org.apache.hadoop.hbase.mob.MobFileCompactionChore;
175import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
176import org.apache.hadoop.hbase.monitoring.MonitoredTask;
177import org.apache.hadoop.hbase.monitoring.TaskMonitor;
178import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
179import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
180import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
181import org.apache.hadoop.hbase.procedure2.LockedResource;
182import org.apache.hadoop.hbase.procedure2.Procedure;
183import org.apache.hadoop.hbase.procedure2.ProcedureEvent;
184import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
185import org.apache.hadoop.hbase.procedure2.RemoteProcedureDispatcher.RemoteProcedure;
186import org.apache.hadoop.hbase.procedure2.RemoteProcedureException;
187import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
188import org.apache.hadoop.hbase.procedure2.store.ProcedureStore.ProcedureStoreListener;
189import org.apache.hadoop.hbase.procedure2.store.region.RegionProcedureStore;
190import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
191import org.apache.hadoop.hbase.quotas.MasterQuotasObserver;
192import org.apache.hadoop.hbase.quotas.QuotaObserverChore;
193import org.apache.hadoop.hbase.quotas.QuotaTableUtil;
194import org.apache.hadoop.hbase.quotas.QuotaUtil;
195import org.apache.hadoop.hbase.quotas.SnapshotQuotaObserverChore;
196import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot;
197import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshot.SpaceQuotaStatus;
198import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifier;
199import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotNotifierFactory;
200import org.apache.hadoop.hbase.quotas.SpaceViolationPolicy;
201import org.apache.hadoop.hbase.regionserver.HRegionServer;
202import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
203import org.apache.hadoop.hbase.regionserver.storefiletracker.ModifyColumnFamilyStoreFileTrackerProcedure;
204import org.apache.hadoop.hbase.regionserver.storefiletracker.ModifyTableStoreFileTrackerProcedure;
205import org.apache.hadoop.hbase.replication.ReplicationException;
206import org.apache.hadoop.hbase.replication.ReplicationLoadSource;
207import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
208import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
209import org.apache.hadoop.hbase.replication.ReplicationUtils;
210import org.apache.hadoop.hbase.replication.SyncReplicationState;
211import org.apache.hadoop.hbase.replication.master.ReplicationHFileCleaner;
212import org.apache.hadoop.hbase.replication.master.ReplicationLogCleaner;
213import org.apache.hadoop.hbase.rsgroup.RSGroupAdminEndpoint;
214import org.apache.hadoop.hbase.rsgroup.RSGroupBasedLoadBalancer;
215import org.apache.hadoop.hbase.rsgroup.RSGroupInfoManager;
216import org.apache.hadoop.hbase.rsgroup.RSGroupUtil;
217import org.apache.hadoop.hbase.security.AccessDeniedException;
218import org.apache.hadoop.hbase.security.SecurityConstants;
219import org.apache.hadoop.hbase.security.Superusers;
220import org.apache.hadoop.hbase.security.UserProvider;
221import org.apache.hadoop.hbase.util.Addressing;
222import org.apache.hadoop.hbase.util.Bytes;
223import org.apache.hadoop.hbase.util.CommonFSUtils;
224import org.apache.hadoop.hbase.util.CoprocessorConfigurationUtil;
225import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
226import org.apache.hadoop.hbase.util.FSTableDescriptors;
227import org.apache.hadoop.hbase.util.FutureUtils;
228import org.apache.hadoop.hbase.util.HBaseFsck;
229import org.apache.hadoop.hbase.util.HFileArchiveUtil;
230import org.apache.hadoop.hbase.util.IdLock;
231import org.apache.hadoop.hbase.util.JVMClusterUtil;
232import org.apache.hadoop.hbase.util.ModifyRegionUtils;
233import org.apache.hadoop.hbase.util.Pair;
234import org.apache.hadoop.hbase.util.RetryCounter;
235import org.apache.hadoop.hbase.util.RetryCounterFactory;
236import org.apache.hadoop.hbase.util.TableDescriptorChecker;
237import org.apache.hadoop.hbase.util.Threads;
238import org.apache.hadoop.hbase.util.VersionInfo;
239import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
240import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
241import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
242import org.apache.hadoop.hbase.zookeeper.RegionNormalizerTracker;
243import org.apache.hadoop.hbase.zookeeper.SnapshotCleanupTracker;
244import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
245import org.apache.hadoop.hbase.zookeeper.ZKUtil;
246import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
247import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
248import org.apache.yetus.audience.InterfaceAudience;
249import org.apache.zookeeper.KeeperException;
250import org.slf4j.Logger;
251import org.slf4j.LoggerFactory;
252
253import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
254import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
255import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
256import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
257import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
258import org.apache.hbase.thirdparty.com.google.protobuf.Service;
259import org.apache.hbase.thirdparty.org.eclipse.jetty.server.Server;
260import org.apache.hbase.thirdparty.org.eclipse.jetty.server.ServerConnector;
261import org.apache.hbase.thirdparty.org.eclipse.jetty.servlet.ServletHolder;
262import org.apache.hbase.thirdparty.org.eclipse.jetty.webapp.WebAppContext;
263import org.apache.hbase.thirdparty.org.glassfish.jersey.server.ResourceConfig;
264import org.apache.hbase.thirdparty.org.glassfish.jersey.servlet.ServletContainer;
265
266import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
267import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.GetRegionInfoResponse;
268import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
269
270/**
271 * HMaster is the "master server" for HBase. An HBase cluster has one active master. If many masters
272 * are started, all compete. Whichever wins goes on to run the cluster. All others park themselves
273 * in their constructor until master or cluster shutdown or until the active master loses its lease
274 * in zookeeper. Thereafter, all running master jostle to take over master role.
275 * <p/>
276 * The Master can be asked shutdown the cluster. See {@link #shutdown()}. In this case it will tell
277 * all regionservers to go down and then wait on them all reporting in that they are down. This
278 * master will then shut itself down.
279 * <p/>
280 * You can also shutdown just this master. Call {@link #stopMaster()}.
281 * @see org.apache.zookeeper.Watcher
282 */
283@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
284public class HMaster extends HBaseServerBase<MasterRpcServices> implements MasterServices {
285
286  private static final Logger LOG = LoggerFactory.getLogger(HMaster.class);
287
288  // MASTER is name of the webapp and the attribute name used stuffing this
289  // instance into a web context !! AND OTHER PLACES !!
290  public static final String MASTER = "master";
291
292  // Manager and zk listener for master election
293  private final ActiveMasterManager activeMasterManager;
294  // Region server tracker
295  private final RegionServerTracker regionServerTracker;
296  // Draining region server tracker
297  private DrainingServerTracker drainingServerTracker;
298  // Tracker for load balancer state
299  LoadBalancerTracker loadBalancerTracker;
300  // Tracker for meta location, if any client ZK quorum specified
301  private MetaLocationSyncer metaLocationSyncer;
302  // Tracker for active master location, if any client ZK quorum specified
303  @InterfaceAudience.Private
304  MasterAddressSyncer masterAddressSyncer;
305  // Tracker for auto snapshot cleanup state
306  SnapshotCleanupTracker snapshotCleanupTracker;
307
308  // Tracker for split and merge state
309  private SplitOrMergeTracker splitOrMergeTracker;
310
311  private ClusterSchemaService clusterSchemaService;
312
313  public static final String HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS =
314    "hbase.master.wait.on.service.seconds";
315  public static final int DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS = 5 * 60;
316
317  public static final String HBASE_MASTER_CLEANER_INTERVAL = "hbase.master.cleaner.interval";
318
319  public static final int DEFAULT_HBASE_MASTER_CLEANER_INTERVAL = 600 * 1000;
320
321  private String clusterId;
322
323  // Metrics for the HMaster
324  final MetricsMaster metricsMaster;
325  // file system manager for the master FS operations
326  private MasterFileSystem fileSystemManager;
327  private MasterWalManager walManager;
328
329  // manager to manage procedure-based WAL splitting, can be null if current
330  // is zk-based WAL splitting. SplitWALManager will replace SplitLogManager
331  // and MasterWalManager, which means zk-based WAL splitting code will be
332  // useless after we switch to the procedure-based one. our eventual goal
333  // is to remove all the zk-based WAL splitting code.
334  private SplitWALManager splitWALManager;
335
336  // server manager to deal with region server info
337  private volatile ServerManager serverManager;
338
339  // manager of assignment nodes in zookeeper
340  private AssignmentManager assignmentManager;
341
342  private RSGroupInfoManager rsGroupInfoManager;
343
344  // manager of replication
345  private ReplicationPeerManager replicationPeerManager;
346
347  private SyncReplicationReplayWALManager syncReplicationReplayWALManager;
348
349  // buffer for "fatal error" notices from region servers
350  // in the cluster. This is only used for assisting
351  // operations/debugging.
352  MemoryBoundedLogMessageBuffer rsFatals;
353
354  // flag set after we become the active master (used for testing)
355  private volatile boolean activeMaster = false;
356
357  // flag set after we complete initialization once active
358  private final ProcedureEvent<?> initialized = new ProcedureEvent<>("master initialized");
359
360  // flag set after master services are started,
361  // initialization may have not completed yet.
362  volatile boolean serviceStarted = false;
363
364  // Maximum time we should run balancer for
365  private final int maxBalancingTime;
366  // Maximum percent of regions in transition when balancing
367  private final double maxRitPercent;
368
369  private final LockManager lockManager = new LockManager(this);
370
371  private RSGroupBasedLoadBalancer balancer;
372  private BalancerChore balancerChore;
373  private static boolean disableBalancerChoreForTest = false;
374  private RegionNormalizerManager regionNormalizerManager;
375  private ClusterStatusChore clusterStatusChore;
376  private ClusterStatusPublisher clusterStatusPublisherChore = null;
377  private SnapshotCleanerChore snapshotCleanerChore = null;
378
379  private HbckChore hbckChore;
380  CatalogJanitor catalogJanitorChore;
381  // Threadpool for scanning the archive directory, used by the HFileCleaner
382  private DirScanPool hfileCleanerPool;
383  // Threadpool for scanning the Old logs directory, used by the LogCleaner
384  private DirScanPool logCleanerPool;
385  private LogCleaner logCleaner;
386  private HFileCleaner hfileCleaner;
387  private ReplicationBarrierCleaner replicationBarrierCleaner;
388  private MobFileCleanerChore mobFileCleanerChore;
389  private MobFileCompactionChore mobFileCompactionChore;
390  private RollingUpgradeChore rollingUpgradeChore;
391  // used to synchronize the mobCompactionStates
392  private final IdLock mobCompactionLock = new IdLock();
393  // save the information of mob compactions in tables.
394  // the key is table name, the value is the number of compactions in that table.
395  private Map<TableName, AtomicInteger> mobCompactionStates = Maps.newConcurrentMap();
396
397  volatile MasterCoprocessorHost cpHost;
398
399  private final boolean preLoadTableDescriptors;
400
401  // Time stamps for when a hmaster became active
402  private long masterActiveTime;
403
404  // Time stamp for when HMaster finishes becoming Active Master
405  private long masterFinishedInitializationTime;
406
407  Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
408
409  // monitor for snapshot of hbase tables
410  SnapshotManager snapshotManager;
411  // monitor for distributed procedures
412  private MasterProcedureManagerHost mpmHost;
413
414  private RegionsRecoveryChore regionsRecoveryChore = null;
415
416  private RegionsRecoveryConfigManager regionsRecoveryConfigManager = null;
417  // it is assigned after 'initialized' guard set to true, so should be volatile
418  private volatile MasterQuotaManager quotaManager;
419  private SpaceQuotaSnapshotNotifier spaceQuotaSnapshotNotifier;
420  private QuotaObserverChore quotaObserverChore;
421  private SnapshotQuotaObserverChore snapshotQuotaChore;
422
423  private ProcedureExecutor<MasterProcedureEnv> procedureExecutor;
424  private ProcedureStore procedureStore;
425
426  // the master local storage to store procedure data, meta region locations, etc.
427  private MasterRegion masterRegion;
428
429  private RegionServerList rsListStorage;
430
431  // handle table states
432  private TableStateManager tableStateManager;
433
434  /** jetty server for master to redirect requests to regionserver infoServer */
435  private Server masterJettyServer;
436
437  // Determine if we should do normal startup or minimal "single-user" mode with no region
438  // servers and no user tables. Useful for repair and recovery of hbase:meta
439  private final boolean maintenanceMode;
440  static final String MAINTENANCE_MODE = "hbase.master.maintenance_mode";
441
442  // the in process region server for carry system regions in maintenanceMode
443  private JVMClusterUtil.RegionServerThread maintenanceRegionServer;
444
445  // Cached clusterId on stand by masters to serve clusterID requests from clients.
446  private final CachedClusterId cachedClusterId;
447
448  public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move";
449  private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true;
450
451  /**
452   * Initializes the HMaster. The steps are as follows:
453   * <p>
454   * <ol>
455   * <li>Initialize the local HRegionServer
456   * <li>Start the ActiveMasterManager.
457   * </ol>
458   * <p>
459   * Remaining steps of initialization occur in
460   * {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
461   * one.
462   */
463  public HMaster(final Configuration conf) throws IOException {
464    super(conf, "Master");
465    try {
466      if (conf.getBoolean(MAINTENANCE_MODE, false)) {
467        LOG.info("Detected {}=true via configuration.", MAINTENANCE_MODE);
468        maintenanceMode = true;
469      } else if (Boolean.getBoolean(MAINTENANCE_MODE)) {
470        LOG.info("Detected {}=true via environment variables.", MAINTENANCE_MODE);
471        maintenanceMode = true;
472      } else {
473        maintenanceMode = false;
474      }
475      this.rsFatals = new MemoryBoundedLogMessageBuffer(
476        conf.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024));
477      LOG.info("hbase.rootdir={}, hbase.cluster.distributed={}",
478        CommonFSUtils.getRootDir(this.conf),
479        this.conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, false));
480
481      // Disable usage of meta replicas in the master
482      this.conf.setBoolean(HConstants.USE_META_REPLICAS, false);
483
484      decorateMasterConfiguration(this.conf);
485
486      // Hack! Maps DFSClient => Master for logs. HDFS made this
487      // config param for task trackers, but we can piggyback off of it.
488      if (this.conf.get("mapreduce.task.attempt.id") == null) {
489        this.conf.set("mapreduce.task.attempt.id", "hb_m_" + this.serverName.toString());
490      }
491
492      this.metricsMaster = new MetricsMaster(new MetricsMasterWrapperImpl(this));
493
494      // preload table descriptor at startup
495      this.preLoadTableDescriptors = conf.getBoolean("hbase.master.preload.tabledescriptors", true);
496
497      this.maxBalancingTime = getMaxBalancingTime();
498      this.maxRitPercent = conf.getDouble(HConstants.HBASE_MASTER_BALANCER_MAX_RIT_PERCENT,
499        HConstants.DEFAULT_HBASE_MASTER_BALANCER_MAX_RIT_PERCENT);
500
501      // Do we publish the status?
502
503      boolean shouldPublish =
504        conf.getBoolean(HConstants.STATUS_PUBLISHED, HConstants.STATUS_PUBLISHED_DEFAULT);
505      Class<? extends ClusterStatusPublisher.Publisher> publisherClass =
506        conf.getClass(ClusterStatusPublisher.STATUS_PUBLISHER_CLASS,
507          ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS,
508          ClusterStatusPublisher.Publisher.class);
509
510      if (shouldPublish) {
511        if (publisherClass == null) {
512          LOG.warn(HConstants.STATUS_PUBLISHED + " is true, but "
513            + ClusterStatusPublisher.DEFAULT_STATUS_PUBLISHER_CLASS
514            + " is not set - not publishing status");
515        } else {
516          clusterStatusPublisherChore = new ClusterStatusPublisher(this, conf, publisherClass);
517          LOG.debug("Created {}", this.clusterStatusPublisherChore);
518          getChoreService().scheduleChore(clusterStatusPublisherChore);
519        }
520      }
521      this.activeMasterManager = createActiveMasterManager(zooKeeper, serverName, this);
522      cachedClusterId = new CachedClusterId(this, conf);
523      this.regionServerTracker = new RegionServerTracker(zooKeeper, this);
524      this.rpcServices.start(zooKeeper);
525    } catch (Throwable t) {
526      // Make sure we log the exception. HMaster is often started via reflection and the
527      // cause of failed startup is lost.
528      LOG.error("Failed construction of Master", t);
529      throw t;
530    }
531  }
532
533  /**
534   * Protected to have custom implementations in tests override the default ActiveMaster
535   * implementation.
536   */
537  protected ActiveMasterManager createActiveMasterManager(ZKWatcher zk, ServerName sn,
538    org.apache.hadoop.hbase.Server server) throws InterruptedIOException {
539    return new ActiveMasterManager(zk, sn, server);
540  }
541
542  @Override
543  protected String getUseThisHostnameInstead(Configuration conf) {
544    return conf.get(MASTER_HOSTNAME_KEY);
545  }
546
547  private void registerConfigurationObservers() {
548    configurationManager.registerObserver(this.rpcServices);
549    configurationManager.registerObserver(this);
550  }
551
552  // Main run loop. Calls through to the regionserver run loop AFTER becoming active Master; will
553  // block in here until then.
554  @Override
555  public void run() {
556    try {
557      installShutdownHook();
558      registerConfigurationObservers();
559      Threads.setDaemonThreadRunning(new Thread(() -> {
560        try {
561          int infoPort = putUpJettyServer();
562          startActiveMasterManager(infoPort);
563        } catch (Throwable t) {
564          // Make sure we log the exception.
565          String error = "Failed to become Active Master";
566          LOG.error(error, t);
567          // Abort should have been called already.
568          if (!isAborted()) {
569            abort(error, t);
570          }
571        }
572      }), getName() + ":becomeActiveMaster");
573      while (!isStopped() && !isAborted()) {
574        sleeper.sleep();
575      }
576      stopInfoServer();
577      closeClusterConnection();
578      stopServiceThreads();
579      if (this.rpcServices != null) {
580        this.rpcServices.stop();
581      }
582      closeZooKeeper();
583    } finally {
584      if (this.clusterSchemaService != null) {
585        // If on way out, then we are no longer active master.
586        this.clusterSchemaService.stopAsync();
587        try {
588          this.clusterSchemaService
589            .awaitTerminated(getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
590              DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);
591        } catch (TimeoutException te) {
592          LOG.warn("Failed shutdown of clusterSchemaService", te);
593        }
594      }
595      this.activeMaster = false;
596    }
597  }
598
599  // return the actual infoPort, -1 means disable info server.
600  private int putUpJettyServer() throws IOException {
601    if (!conf.getBoolean("hbase.master.infoserver.redirect", true)) {
602      return -1;
603    }
604    final int infoPort =
605      conf.getInt("hbase.master.info.port.orig", HConstants.DEFAULT_MASTER_INFOPORT);
606    // -1 is for disabling info server, so no redirecting
607    if (infoPort < 0 || infoServer == null) {
608      return -1;
609    }
610    if (infoPort == infoServer.getPort()) {
611      // server is already running
612      return infoPort;
613    }
614    final String addr = conf.get("hbase.master.info.bindAddress", "0.0.0.0");
615    if (!Addressing.isLocalAddress(InetAddress.getByName(addr))) {
616      String msg = "Failed to start redirecting jetty server. Address " + addr
617        + " does not belong to this host. Correct configuration parameter: "
618        + "hbase.master.info.bindAddress";
619      LOG.error(msg);
620      throw new IOException(msg);
621    }
622
623    // TODO I'm pretty sure we could just add another binding to the InfoServer run by
624    // the RegionServer and have it run the RedirectServlet instead of standing up
625    // a second entire stack here.
626    masterJettyServer = new Server();
627    final ServerConnector connector = new ServerConnector(masterJettyServer);
628    connector.setHost(addr);
629    connector.setPort(infoPort);
630    masterJettyServer.addConnector(connector);
631    masterJettyServer.setStopAtShutdown(true);
632    masterJettyServer.setHandler(HttpServer.buildGzipHandler(masterJettyServer.getHandler()));
633
634    final String redirectHostname =
635      StringUtils.isBlank(useThisHostnameInstead) ? null : useThisHostnameInstead;
636
637    final MasterRedirectServlet redirect = new MasterRedirectServlet(infoServer, redirectHostname);
638    final WebAppContext context =
639      new WebAppContext(null, "/", null, null, null, null, WebAppContext.NO_SESSIONS);
640    context.addServlet(new ServletHolder(redirect), "/*");
641    context.setServer(masterJettyServer);
642
643    try {
644      masterJettyServer.start();
645    } catch (Exception e) {
646      throw new IOException("Failed to start redirecting jetty server", e);
647    }
648    return connector.getLocalPort();
649  }
650
651  /**
652   * For compatibility, if failed with regionserver credentials, try the master one
653   */
654  @Override
655  protected void login(UserProvider user, String host) throws IOException {
656    try {
657      user.login(SecurityConstants.REGIONSERVER_KRB_KEYTAB_FILE,
658        SecurityConstants.REGIONSERVER_KRB_PRINCIPAL, host);
659    } catch (IOException ie) {
660      user.login(SecurityConstants.MASTER_KRB_KEYTAB_FILE, SecurityConstants.MASTER_KRB_PRINCIPAL,
661        host);
662    }
663  }
664
665  public MasterRpcServices getMasterRpcServices() {
666    return rpcServices;
667  }
668
669  public boolean balanceSwitch(final boolean b) throws IOException {
670    return getMasterRpcServices().switchBalancer(b, BalanceSwitchMode.ASYNC);
671  }
672
673  @Override
674  protected String getProcessName() {
675    return MASTER;
676  }
677
678  @Override
679  protected boolean canCreateBaseZNode() {
680    return true;
681  }
682
683  @Override
684  protected boolean canUpdateTableDescriptor() {
685    return true;
686  }
687
688  @Override
689  protected boolean cacheTableDescriptor() {
690    return true;
691  }
692
693  protected MasterRpcServices createRpcServices() throws IOException {
694    return new MasterRpcServices(this);
695  }
696
697  @Override
698  protected void configureInfoServer(InfoServer infoServer) {
699    infoServer.addUnprivilegedServlet("master-status", "/master-status", MasterStatusServlet.class);
700    infoServer.addUnprivilegedServlet("api_v1", "/api/v1/*", buildApiV1Servlet());
701
702    infoServer.setAttribute(MASTER, this);
703  }
704
705  private ServletHolder buildApiV1Servlet() {
706    final ResourceConfig config = ResourceConfigFactory.createResourceConfig(conf, this);
707    return new ServletHolder(new ServletContainer(config));
708  }
709
710  @Override
711  protected Class<? extends HttpServlet> getDumpServlet() {
712    return MasterDumpServlet.class;
713  }
714
715  @Override
716  public MetricsMaster getMasterMetrics() {
717    return metricsMaster;
718  }
719
720  /**
721   * Initialize all ZK based system trackers. But do not include {@link RegionServerTracker}, it
722   * should have already been initialized along with {@link ServerManager}.
723   */
724  private void initializeZKBasedSystemTrackers()
725    throws IOException, KeeperException, ReplicationException {
726    if (maintenanceMode) {
727      // in maintenance mode, always use MaintenanceLoadBalancer.
728      conf.unset(LoadBalancer.HBASE_RSGROUP_LOADBALANCER_CLASS);
729      conf.setClass(HConstants.HBASE_MASTER_LOADBALANCER_CLASS, MaintenanceLoadBalancer.class,
730        LoadBalancer.class);
731    }
732    this.balancer = new RSGroupBasedLoadBalancer();
733    this.loadBalancerTracker = new LoadBalancerTracker(zooKeeper, this);
734    this.loadBalancerTracker.start();
735
736    this.regionNormalizerManager =
737      RegionNormalizerFactory.createNormalizerManager(conf, zooKeeper, this);
738    this.configurationManager.registerObserver(regionNormalizerManager);
739    this.regionNormalizerManager.start();
740
741    this.splitOrMergeTracker = new SplitOrMergeTracker(zooKeeper, conf, this);
742    this.splitOrMergeTracker.start();
743
744    // This is for backwards compatible. We do not need the CP for rs group now but if user want to
745    // load it, we need to enable rs group.
746    String[] cpClasses = conf.getStrings(MasterCoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
747    if (cpClasses != null) {
748      for (String cpClass : cpClasses) {
749        if (RSGroupAdminEndpoint.class.getName().equals(cpClass)) {
750          RSGroupUtil.enableRSGroup(conf);
751          break;
752        }
753      }
754    }
755    this.rsGroupInfoManager = RSGroupInfoManager.create(this);
756
757    this.replicationPeerManager = ReplicationPeerManager.create(zooKeeper, conf, clusterId);
758
759    this.drainingServerTracker = new DrainingServerTracker(zooKeeper, this, this.serverManager);
760    this.drainingServerTracker.start();
761
762    this.snapshotCleanupTracker = new SnapshotCleanupTracker(zooKeeper, this);
763    this.snapshotCleanupTracker.start();
764
765    String clientQuorumServers = conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM);
766    boolean clientZkObserverMode = conf.getBoolean(HConstants.CLIENT_ZOOKEEPER_OBSERVER_MODE,
767      HConstants.DEFAULT_CLIENT_ZOOKEEPER_OBSERVER_MODE);
768    if (clientQuorumServers != null && !clientZkObserverMode) {
769      // we need to take care of the ZK information synchronization
770      // if given client ZK are not observer nodes
771      ZKWatcher clientZkWatcher = new ZKWatcher(conf,
772        getProcessName() + ":" + rpcServices.getSocketAddress().getPort() + "-clientZK", this,
773        false, true);
774      this.metaLocationSyncer = new MetaLocationSyncer(zooKeeper, clientZkWatcher, this);
775      this.metaLocationSyncer.start();
776      this.masterAddressSyncer = new MasterAddressSyncer(zooKeeper, clientZkWatcher, this);
777      this.masterAddressSyncer.start();
778      // set cluster id is a one-go effort
779      ZKClusterId.setClusterId(clientZkWatcher, fileSystemManager.getClusterId());
780    }
781
782    // Set the cluster as up. If new RSs, they'll be waiting on this before
783    // going ahead with their startup.
784    boolean wasUp = this.clusterStatusTracker.isClusterUp();
785    if (!wasUp) this.clusterStatusTracker.setClusterUp();
786
787    LOG.info("Active/primary master=" + this.serverName + ", sessionid=0x"
788      + Long.toHexString(this.zooKeeper.getRecoverableZooKeeper().getSessionId())
789      + ", setting cluster-up flag (Was=" + wasUp + ")");
790
791    // create/initialize the snapshot manager and other procedure managers
792    this.snapshotManager = new SnapshotManager();
793    this.mpmHost = new MasterProcedureManagerHost();
794    this.mpmHost.register(this.snapshotManager);
795    this.mpmHost.register(new MasterFlushTableProcedureManager());
796    this.mpmHost.loadProcedures(conf);
797    this.mpmHost.initialize(this, this.metricsMaster);
798  }
799
800  // Will be overriden in test to inject customized AssignmentManager
801  @InterfaceAudience.Private
802  protected AssignmentManager createAssignmentManager(MasterServices master,
803    MasterRegion masterRegion) {
804    return new AssignmentManager(master, masterRegion);
805  }
806
807  private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperException {
808    // try migrate data from zookeeper
809    try (ResultScanner scanner =
810      masterRegion.getScanner(new Scan().addFamily(HConstants.CATALOG_FAMILY))) {
811      if (scanner.next() != null) {
812        // notice that all replicas for a region are in the same row, so the migration can be
813        // done with in a one row put, which means if we have data in catalog family then we can
814        // make sure that the migration is done.
815        LOG.info("The {} family in master local region already has data in it, skip migrating...",
816          HConstants.CATALOG_FAMILY_STR);
817        return;
818      }
819    }
820    // start migrating
821    byte[] row = CatalogFamilyFormat.getMetaKeyForRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);
822    Put put = new Put(row);
823    List<String> metaReplicaNodes = zooKeeper.getMetaReplicaNodes();
824    StringBuilder info = new StringBuilder("Migrating meta locations:");
825    for (String metaReplicaNode : metaReplicaNodes) {
826      int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZNode(metaReplicaNode);
827      RegionState state = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
828      info.append(" ").append(state);
829      put.setTimestamp(state.getStamp());
830      MetaTableAccessor.addRegionInfo(put, state.getRegion());
831      if (state.getServerName() != null) {
832        MetaTableAccessor.addLocation(put, state.getServerName(), HConstants.NO_SEQNUM, replicaId);
833      }
834      put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
835        .setFamily(HConstants.CATALOG_FAMILY)
836        .setQualifier(RegionStateStore.getStateColumn(replicaId)).setTimestamp(put.getTimestamp())
837        .setType(Cell.Type.Put).setValue(Bytes.toBytes(state.getState().name())).build());
838    }
839    if (!put.isEmpty()) {
840      LOG.info(info.toString());
841      masterRegion.update(r -> r.put(put));
842    } else {
843      LOG.info("No meta location available on zookeeper, skip migrating...");
844    }
845  }
846
847  /**
848   * Finish initialization of HMaster after becoming the primary master.
849   * <p/>
850   * The startup order is a bit complicated but very important, do not change it unless you know
851   * what you are doing.
852   * <ol>
853   * <li>Initialize file system based components - file system manager, wal manager, table
854   * descriptors, etc</li>
855   * <li>Publish cluster id</li>
856   * <li>Here comes the most complicated part - initialize server manager, assignment manager and
857   * region server tracker
858   * <ol type='i'>
859   * <li>Create server manager</li>
860   * <li>Create master local region</li>
861   * <li>Create procedure executor, load the procedures, but do not start workers. We will start it
862   * later after we finish scheduling SCPs to avoid scheduling duplicated SCPs for the same
863   * server</li>
864   * <li>Create assignment manager and start it, load the meta region state, but do not load data
865   * from meta region</li>
866   * <li>Start region server tracker, construct the online servers set and find out dead servers and
867   * schedule SCP for them. The online servers will be constructed by scanning zk, and we will also
868   * scan the wal directory and load from master local region to find out possible live region
869   * servers, and the differences between these two sets are the dead servers</li>
870   * </ol>
871   * </li>
872   * <li>If this is a new deploy, schedule a InitMetaProcedure to initialize meta</li>
873   * <li>Start necessary service threads - balancer, catalog janitor, executor services, and also
874   * the procedure executor, etc. Notice that the balancer must be created first as assignment
875   * manager may use it when assigning regions.</li>
876   * <li>Wait for meta to be initialized if necessary, start table state manager.</li>
877   * <li>Wait for enough region servers to check-in</li>
878   * <li>Let assignment manager load data from meta and construct region states</li>
879   * <li>Start all other things such as chore services, etc</li>
880   * </ol>
881   * <p/>
882   * Notice that now we will not schedule a special procedure to make meta online(unless the first
883   * time where meta has not been created yet), we will rely on SCP to bring meta online.
884   */
885  private void finishActiveMasterInitialization(MonitoredTask status)
886    throws IOException, InterruptedException, KeeperException, ReplicationException {
887    /*
888     * We are active master now... go initialize components we need to run.
889     */
890    status.setStatus("Initializing Master file system");
891
892    this.masterActiveTime = EnvironmentEdgeManager.currentTime();
893    // TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
894
895    // always initialize the MemStoreLAB as we use a region to store data in master now, see
896    // localStore.
897    initializeMemStoreChunkCreator(null);
898    this.fileSystemManager = new MasterFileSystem(conf);
899    this.walManager = new MasterWalManager(this);
900
901    // warm-up HTDs cache on master initialization
902    if (preLoadTableDescriptors) {
903      status.setStatus("Pre-loading table descriptors");
904      this.tableDescriptors.getAll();
905    }
906
907    // Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
908    // only after it has checked in with the Master. At least a few tests ask Master for clusterId
909    // before it has called its run method and before RegionServer has done the reportForDuty.
910    ClusterId clusterId = fileSystemManager.getClusterId();
911    status.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
912    ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
913    this.clusterId = clusterId.toString();
914
915    // Precaution. Put in place the old hbck1 lock file to fence out old hbase1s running their
916    // hbck1s against an hbase2 cluster; it could do damage. To skip this behavior, set
917    // hbase.write.hbck1.lock.file to false.
918    if (this.conf.getBoolean("hbase.write.hbck1.lock.file", true)) {
919      Pair<Path, FSDataOutputStream> result = null;
920      try {
921        result = HBaseFsck.checkAndMarkRunningHbck(this.conf,
922          HBaseFsck.createLockRetryCounterFactory(this.conf).create());
923      } finally {
924        if (result != null) {
925          Closeables.close(result.getSecond(), true);
926        }
927      }
928    }
929
930    status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
931    // The below two managers must be created before loading procedures, as they will be used during
932    // loading.
933    // initialize master local region
934    masterRegion = MasterRegionFactory.create(this);
935    rsListStorage = new MasterRegionServerList(masterRegion, this);
936
937    this.serverManager = createServerManager(this, rsListStorage);
938    this.syncReplicationReplayWALManager = new SyncReplicationReplayWALManager(this);
939    if (
940      !conf.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK, DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)
941    ) {
942      this.splitWALManager = new SplitWALManager(this);
943    }
944
945    tryMigrateMetaLocationsFromZooKeeper();
946
947    createProcedureExecutor();
948    Map<Class<?>, List<Procedure<MasterProcedureEnv>>> procsByType = procedureExecutor
949      .getActiveProceduresNoCopy().stream().collect(Collectors.groupingBy(p -> p.getClass()));
950
951    // Create Assignment Manager
952    this.assignmentManager = createAssignmentManager(this, masterRegion);
953    this.assignmentManager.start();
954    // TODO: TRSP can perform as the sub procedure for other procedures, so even if it is marked as
955    // completed, it could still be in the procedure list. This is a bit strange but is another
956    // story, need to verify the implementation for ProcedureExecutor and ProcedureStore.
957    List<TransitRegionStateProcedure> ritList =
958      procsByType.getOrDefault(TransitRegionStateProcedure.class, Collections.emptyList()).stream()
959        .filter(p -> !p.isFinished()).map(p -> (TransitRegionStateProcedure) p)
960        .collect(Collectors.toList());
961    this.assignmentManager.setupRIT(ritList);
962
963    // Start RegionServerTracker with listing of servers found with exiting SCPs -- these should
964    // be registered in the deadServers set -- and the servernames loaded from the WAL directory
965    // and master local region that COULD BE 'alive'(we'll schedule SCPs for each and let SCP figure
966    // it out).
967    // We also pass dirs that are already 'splitting'... so we can do some checks down in tracker.
968    // TODO: Generate the splitting and live Set in one pass instead of two as we currently do.
969    this.regionServerTracker.upgrade(
970      procsByType.getOrDefault(ServerCrashProcedure.class, Collections.emptyList()).stream()
971        .map(p -> (ServerCrashProcedure) p).map(p -> p.getServerName()).collect(Collectors.toSet()),
972      Sets.union(rsListStorage.getAll(), walManager.getLiveServersFromWALDir()),
973      walManager.getSplittingServersFromWALDir());
974    // This manager must be accessed AFTER hbase:meta is confirmed on line..
975    this.tableStateManager = new TableStateManager(this);
976
977    status.setStatus("Initializing ZK system trackers");
978    initializeZKBasedSystemTrackers();
979    status.setStatus("Loading last flushed sequence id of regions");
980    try {
981      this.serverManager.loadLastFlushedSequenceIds();
982    } catch (IOException e) {
983      LOG.info("Failed to load last flushed sequence id of regions" + " from file system", e);
984    }
985    // Set ourselves as active Master now our claim has succeeded up in zk.
986    this.activeMaster = true;
987
988    // Start the Zombie master detector after setting master as active, see HBASE-21535
989    Thread zombieDetector = new Thread(new MasterInitializationMonitor(this),
990      "ActiveMasterInitializationMonitor-" + EnvironmentEdgeManager.currentTime());
991    zombieDetector.setDaemon(true);
992    zombieDetector.start();
993
994    if (!maintenanceMode) {
995      status.setStatus("Initializing master coprocessors");
996      setQuotasObserver(conf);
997      initializeCoprocessorHost(conf);
998    } else {
999      // start an in process region server for carrying system regions
1000      maintenanceRegionServer =
1001        JVMClusterUtil.createRegionServerThread(getConfiguration(), HRegionServer.class, 0);
1002      maintenanceRegionServer.start();
1003    }
1004
1005    // Checking if meta needs initializing.
1006    status.setStatus("Initializing meta table if this is a new deploy");
1007    InitMetaProcedure initMetaProc = null;
1008    // Print out state of hbase:meta on startup; helps debugging.
1009    if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) {
1010      Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
1011        .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
1012      initMetaProc = optProc.orElseGet(() -> {
1013        // schedule an init meta procedure if meta has not been deployed yet
1014        InitMetaProcedure temp = new InitMetaProcedure();
1015        procedureExecutor.submitProcedure(temp);
1016        return temp;
1017      });
1018    }
1019
1020    // initialize load balancer
1021    this.balancer.setMasterServices(this);
1022    this.balancer.initialize();
1023    this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1024
1025    // start up all service threads.
1026    status.setStatus("Initializing master service threads");
1027    startServiceThreads();
1028    // wait meta to be initialized after we start procedure executor
1029    if (initMetaProc != null) {
1030      initMetaProc.await();
1031    }
1032    // Wake up this server to check in
1033    sleeper.skipSleepCycle();
1034
1035    // Wait for region servers to report in.
1036    // With this as part of master initialization, it precludes our being able to start a single
1037    // server that is both Master and RegionServer. Needs more thought. TODO.
1038    String statusStr = "Wait for region servers to report in";
1039    status.setStatus(statusStr);
1040    LOG.info(Objects.toString(status));
1041    waitForRegionServers(status);
1042
1043    // Check if master is shutting down because issue initializing regionservers or balancer.
1044    if (isStopped()) {
1045      return;
1046    }
1047
1048    status.setStatus("Starting assignment manager");
1049    // FIRST HBASE:META READ!!!!
1050    // The below cannot make progress w/o hbase:meta being online.
1051    // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
1052    // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
1053    // if it is down. It may take a while to come online. So, wait here until meta if for sure
1054    // available. That's what waitForMetaOnline does.
1055    if (!waitForMetaOnline()) {
1056      return;
1057    }
1058
1059    TableDescriptor metaDescriptor = tableDescriptors.get(TableName.META_TABLE_NAME);
1060    final ColumnFamilyDescriptor tableFamilyDesc =
1061      metaDescriptor.getColumnFamily(HConstants.TABLE_FAMILY);
1062    final ColumnFamilyDescriptor replBarrierFamilyDesc =
1063      metaDescriptor.getColumnFamily(HConstants.REPLICATION_BARRIER_FAMILY);
1064
1065    this.assignmentManager.joinCluster();
1066    // The below depends on hbase:meta being online.
1067    this.assignmentManager.processOfflineRegions();
1068    // this must be called after the above processOfflineRegions to prevent race
1069    this.assignmentManager.wakeMetaLoadedEvent();
1070
1071    // for migrating from a version without HBASE-25099, and also for honoring the configuration
1072    // first.
1073    if (conf.get(HConstants.META_REPLICAS_NUM) != null) {
1074      int replicasNumInConf =
1075        conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM);
1076      TableDescriptor metaDesc = tableDescriptors.get(TableName.META_TABLE_NAME);
1077      if (metaDesc.getRegionReplication() != replicasNumInConf) {
1078        // it is possible that we already have some replicas before upgrading, so we must set the
1079        // region replication number in meta TableDescriptor directly first, without creating a
1080        // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas.
1081        int existingReplicasCount =
1082          assignmentManager.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).size();
1083        if (existingReplicasCount > metaDesc.getRegionReplication()) {
1084          LOG.info("Update replica count of hbase:meta from {}(in TableDescriptor)"
1085            + " to {}(existing ZNodes)", metaDesc.getRegionReplication(), existingReplicasCount);
1086          metaDesc = TableDescriptorBuilder.newBuilder(metaDesc)
1087            .setRegionReplication(existingReplicasCount).build();
1088          tableDescriptors.update(metaDesc);
1089        }
1090        // check again, and issue a ModifyTableProcedure if needed
1091        if (metaDesc.getRegionReplication() != replicasNumInConf) {
1092          LOG.info(
1093            "The {} config is {} while the replica count in TableDescriptor is {}"
1094              + " for hbase:meta, altering...",
1095            HConstants.META_REPLICAS_NUM, replicasNumInConf, metaDesc.getRegionReplication());
1096          procedureExecutor.submitProcedure(new ModifyTableProcedure(
1097            procedureExecutor.getEnvironment(), TableDescriptorBuilder.newBuilder(metaDesc)
1098              .setRegionReplication(replicasNumInConf).build(),
1099            null, metaDesc, false));
1100        }
1101      }
1102    }
1103    // Initialize after meta is up as below scans meta
1104    FavoredNodesManager fnm = getFavoredNodesManager();
1105    if (fnm != null) {
1106      fnm.initializeFromMeta();
1107    }
1108
1109    // set cluster status again after user regions are assigned
1110    this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1111
1112    // Start balancer and meta catalog janitor after meta and regions have been assigned.
1113    status.setStatus("Starting balancer and catalog janitor");
1114    this.clusterStatusChore = new ClusterStatusChore(this, balancer);
1115    getChoreService().scheduleChore(clusterStatusChore);
1116    this.balancerChore = new BalancerChore(this);
1117    if (!disableBalancerChoreForTest) {
1118      getChoreService().scheduleChore(balancerChore);
1119    }
1120    if (regionNormalizerManager != null) {
1121      getChoreService().scheduleChore(regionNormalizerManager.getRegionNormalizerChore());
1122    }
1123    this.catalogJanitorChore = new CatalogJanitor(this);
1124    getChoreService().scheduleChore(catalogJanitorChore);
1125    this.hbckChore = new HbckChore(this);
1126    getChoreService().scheduleChore(hbckChore);
1127    this.serverManager.startChore();
1128
1129    // Only for rolling upgrade, where we need to migrate the data in namespace table to meta table.
1130    if (!waitForNamespaceOnline()) {
1131      return;
1132    }
1133    status.setStatus("Starting cluster schema service");
1134    try {
1135      initClusterSchemaService();
1136    } catch (IllegalStateException e) {
1137      if (
1138        e.getCause() != null && e.getCause() instanceof NoSuchColumnFamilyException
1139          && tableFamilyDesc == null && replBarrierFamilyDesc == null
1140      ) {
1141        LOG.info("ClusterSchema service could not be initialized. This is "
1142          + "expected during HBase 1 to 2 upgrade", e);
1143      } else {
1144        throw e;
1145      }
1146    }
1147
1148    if (this.cpHost != null) {
1149      try {
1150        this.cpHost.preMasterInitialization();
1151      } catch (IOException e) {
1152        LOG.error("Coprocessor preMasterInitialization() hook failed", e);
1153      }
1154    }
1155
1156    status.markComplete("Initialization successful");
1157    LOG.info(String.format("Master has completed initialization %.3fsec",
1158      (EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));
1159    this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();
1160    configurationManager.registerObserver(this.balancer);
1161    configurationManager.registerObserver(this.hfileCleanerPool);
1162    configurationManager.registerObserver(this.logCleanerPool);
1163    configurationManager.registerObserver(this.hfileCleaner);
1164    configurationManager.registerObserver(this.logCleaner);
1165    configurationManager.registerObserver(this.regionsRecoveryConfigManager);
1166    // Set master as 'initialized'.
1167    setInitialized(true);
1168
1169    if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
1170      // create missing CFs in meta table after master is set to 'initialized'.
1171      createMissingCFsInMetaDuringUpgrade(metaDescriptor);
1172
1173      // Throwing this Exception to abort active master is painful but this
1174      // seems the only way to add missing CFs in meta while upgrading from
1175      // HBase 1 to 2 (where HBase 2 has HBASE-23055 & HBASE-23782 checked-in).
1176      // So, why do we abort active master after adding missing CFs in meta?
1177      // When we reach here, we would have already bypassed NoSuchColumnFamilyException
1178      // in initClusterSchemaService(), meaning ClusterSchemaService is not
1179      // correctly initialized but we bypassed it. Similarly, we bypassed
1180      // tableStateManager.start() as well. Hence, we should better abort
1181      // current active master because our main task - adding missing CFs
1182      // in meta table is done (possible only after master state is set as
1183      // initialized) at the expense of bypassing few important tasks as part
1184      // of active master init routine. So now we abort active master so that
1185      // next active master init will not face any issues and all mandatory
1186      // services will be started during master init phase.
1187      throw new PleaseRestartMasterException("Aborting active master after missing"
1188        + " CFs are successfully added in meta. Subsequent active master "
1189        + "initialization should be uninterrupted");
1190    }
1191
1192    if (maintenanceMode) {
1193      LOG.info("Detected repair mode, skipping final initialization steps.");
1194      return;
1195    }
1196
1197    assignmentManager.checkIfShouldMoveSystemRegionAsync();
1198    status.setStatus("Starting quota manager");
1199    initQuotaManager();
1200    if (QuotaUtil.isQuotaEnabled(conf)) {
1201      // Create the quota snapshot notifier
1202      spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();
1203      spaceQuotaSnapshotNotifier.initialize(getConnection());
1204      this.quotaObserverChore = new QuotaObserverChore(this, getMasterMetrics());
1205      // Start the chore to read the region FS space reports and act on them
1206      getChoreService().scheduleChore(quotaObserverChore);
1207
1208      this.snapshotQuotaChore = new SnapshotQuotaObserverChore(this, getMasterMetrics());
1209      // Start the chore to read snapshots and add their usage to table/NS quotas
1210      getChoreService().scheduleChore(snapshotQuotaChore);
1211    }
1212    final SlowLogMasterService slowLogMasterService = new SlowLogMasterService(conf, this);
1213    slowLogMasterService.init();
1214
1215    // clear the dead servers with same host name and port of online server because we are not
1216    // removing dead server with same hostname and port of rs which is trying to check in before
1217    // master initialization. See HBASE-5916.
1218    this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
1219
1220    // Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
1221    status.setStatus("Checking ZNode ACLs");
1222    zooKeeper.checkAndSetZNodeAcls();
1223
1224    status.setStatus("Initializing MOB Cleaner");
1225    initMobCleaner();
1226
1227    status.setStatus("Calling postStartMaster coprocessors");
1228    if (this.cpHost != null) {
1229      // don't let cp initialization errors kill the master
1230      try {
1231        this.cpHost.postStartMaster();
1232      } catch (IOException ioe) {
1233        LOG.error("Coprocessor postStartMaster() hook failed", ioe);
1234      }
1235    }
1236
1237    zombieDetector.interrupt();
1238
1239    /*
1240     * After master has started up, lets do balancer post startup initialization. Since this runs in
1241     * activeMasterManager thread, it should be fine.
1242     */
1243    long start = EnvironmentEdgeManager.currentTime();
1244    this.balancer.postMasterStartupInitialize();
1245    if (LOG.isDebugEnabled()) {
1246      LOG.debug("Balancer post startup initialization complete, took "
1247        + ((EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds");
1248    }
1249
1250    this.rollingUpgradeChore = new RollingUpgradeChore(this);
1251    getChoreService().scheduleChore(rollingUpgradeChore);
1252  }
1253
1254  private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
1255    throws IOException {
1256    TableDescriptor newMetaDesc = TableDescriptorBuilder.newBuilder(metaDescriptor)
1257      .setColumnFamily(FSTableDescriptors.getTableFamilyDescForMeta(conf))
1258      .setColumnFamily(FSTableDescriptors.getReplBarrierFamilyDescForMeta()).build();
1259    long pid = this.modifyTable(TableName.META_TABLE_NAME, () -> newMetaDesc, 0, 0, false);
1260    int tries = 30;
1261    while (
1262      !(getMasterProcedureExecutor().isFinished(pid)) && getMasterProcedureExecutor().isRunning()
1263        && tries > 0
1264    ) {
1265      try {
1266        Thread.sleep(1000);
1267      } catch (InterruptedException e) {
1268        throw new IOException("Wait interrupted", e);
1269      }
1270      tries--;
1271    }
1272    if (tries <= 0) {
1273      throw new HBaseIOException(
1274        "Failed to add table and rep_barrier CFs to meta in a given time.");
1275    } else {
1276      Procedure<?> result = getMasterProcedureExecutor().getResult(pid);
1277      if (result != null && result.isFailed()) {
1278        throw new IOException("Failed to add table and rep_barrier CFs to meta. "
1279          + MasterProcedureUtil.unwrapRemoteIOException(result));
1280      }
1281    }
1282  }
1283
1284  /**
1285   * Check hbase:meta is up and ready for reading. For use during Master startup only.
1286   * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
1287   *         and we will hold here until operator intervention.
1288   */
1289  @InterfaceAudience.Private
1290  public boolean waitForMetaOnline() {
1291    return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
1292  }
1293
1294  /**
1295   * @return True if region is online and scannable else false if an error or shutdown (Otherwise we
1296   *         just block in here holding up all forward-progess).
1297   */
1298  private boolean isRegionOnline(RegionInfo ri) {
1299    RetryCounter rc = null;
1300    while (!isStopped()) {
1301      RegionState rs = this.assignmentManager.getRegionStates().getRegionState(ri);
1302      if (rs.isOpened()) {
1303        if (this.getServerManager().isServerOnline(rs.getServerName())) {
1304          return true;
1305        }
1306      }
1307      // Region is not OPEN.
1308      Optional<Procedure<MasterProcedureEnv>> optProc = this.procedureExecutor.getProcedures()
1309        .stream().filter(p -> p instanceof ServerCrashProcedure).findAny();
1310      // TODO: Add a page to refguide on how to do repair. Have this log message point to it.
1311      // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
1312      // then how to assign including how to break region lock if one held.
1313      LOG.warn(
1314        "{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot "
1315          + "progress, in holding-pattern until region onlined.",
1316        ri.getRegionNameAsString(), rs, optProc.isPresent());
1317      // Check once-a-minute.
1318      if (rc == null) {
1319        rc = new RetryCounterFactory(Integer.MAX_VALUE, 1000, 60_000).create();
1320      }
1321      Threads.sleep(rc.getBackoffTimeAndIncrementAttempts());
1322    }
1323    return false;
1324  }
1325
1326  /**
1327   * Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table
1328   * <p/>
1329   * This is for rolling upgrading, later we will migrate the data in ns table to the ns family of
1330   * meta table. And if this is a new cluster, this method will return immediately as there will be
1331   * no namespace table/region.
1332   * @return True if namespace table is up/online.
1333   */
1334  private boolean waitForNamespaceOnline() throws IOException {
1335    TableState nsTableState =
1336      MetaTableAccessor.getTableState(getConnection(), TableName.NAMESPACE_TABLE_NAME);
1337    if (nsTableState == null || nsTableState.isDisabled()) {
1338      // this means we have already migrated the data and disabled or deleted the namespace table,
1339      // or this is a new deploy which does not have a namespace table from the beginning.
1340      return true;
1341    }
1342    List<RegionInfo> ris =
1343      this.assignmentManager.getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
1344    if (ris.isEmpty()) {
1345      // maybe this will not happen any more, but anyway, no harm to add a check here...
1346      return true;
1347    }
1348    // Else there are namespace regions up in meta. Ensure they are assigned before we go on.
1349    for (RegionInfo ri : ris) {
1350      if (!isRegionOnline(ri)) {
1351        return false;
1352      }
1353    }
1354    return true;
1355  }
1356
1357  /**
1358   * Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
1359   * automatically remove quotas for a table when that table is deleted.
1360   */
1361  @InterfaceAudience.Private
1362  public void updateConfigurationForQuotasObserver(Configuration conf) {
1363    // We're configured to not delete quotas on table deletion, so we don't need to add the obs.
1364    if (
1365      !conf.getBoolean(MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE,
1366        MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT)
1367    ) {
1368      return;
1369    }
1370    String[] masterCoprocs = conf.getStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
1371    final int length = null == masterCoprocs ? 0 : masterCoprocs.length;
1372    String[] updatedCoprocs = new String[length + 1];
1373    if (length > 0) {
1374      System.arraycopy(masterCoprocs, 0, updatedCoprocs, 0, masterCoprocs.length);
1375    }
1376    updatedCoprocs[length] = MasterQuotasObserver.class.getName();
1377    conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, updatedCoprocs);
1378  }
1379
1380  private void initMobCleaner() {
1381    this.mobFileCleanerChore = new MobFileCleanerChore(this);
1382    getChoreService().scheduleChore(mobFileCleanerChore);
1383    this.mobFileCompactionChore = new MobFileCompactionChore(this);
1384    getChoreService().scheduleChore(mobFileCompactionChore);
1385  }
1386
1387  /**
1388   * <p>
1389   * Create a {@link ServerManager} instance.
1390   * </p>
1391   * <p>
1392   * Will be overridden in tests.
1393   * </p>
1394   */
1395  @InterfaceAudience.Private
1396  protected ServerManager createServerManager(MasterServices master, RegionServerList storage)
1397    throws IOException {
1398    // We put this out here in a method so can do a Mockito.spy and stub it out
1399    // w/ a mocked up ServerManager.
1400    setupClusterConnection();
1401    return new ServerManager(master, storage);
1402  }
1403
1404  private void waitForRegionServers(final MonitoredTask status)
1405    throws IOException, InterruptedException {
1406    this.serverManager.waitForRegionServers(status);
1407  }
1408
1409  // Will be overridden in tests
1410  @InterfaceAudience.Private
1411  protected void initClusterSchemaService() throws IOException, InterruptedException {
1412    this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
1413    this.clusterSchemaService.startAsync();
1414    try {
1415      this.clusterSchemaService
1416        .awaitRunning(getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
1417          DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);
1418    } catch (TimeoutException toe) {
1419      throw new IOException("Timedout starting ClusterSchemaService", toe);
1420    }
1421  }
1422
1423  private void initQuotaManager() throws IOException {
1424    MasterQuotaManager quotaManager = new MasterQuotaManager(this);
1425    quotaManager.start();
1426    this.quotaManager = quotaManager;
1427  }
1428
1429  private SpaceQuotaSnapshotNotifier createQuotaSnapshotNotifier() {
1430    SpaceQuotaSnapshotNotifier notifier =
1431      SpaceQuotaSnapshotNotifierFactory.getInstance().create(getConfiguration());
1432    return notifier;
1433  }
1434
1435  public boolean isCatalogJanitorEnabled() {
1436    return catalogJanitorChore != null ? catalogJanitorChore.getEnabled() : false;
1437  }
1438
1439  boolean isCleanerChoreEnabled() {
1440    boolean hfileCleanerFlag = true, logCleanerFlag = true;
1441
1442    if (hfileCleaner != null) {
1443      hfileCleanerFlag = hfileCleaner.getEnabled();
1444    }
1445
1446    if (logCleaner != null) {
1447      logCleanerFlag = logCleaner.getEnabled();
1448    }
1449
1450    return (hfileCleanerFlag && logCleanerFlag);
1451  }
1452
1453  @Override
1454  public ServerManager getServerManager() {
1455    return this.serverManager;
1456  }
1457
1458  @Override
1459  public MasterFileSystem getMasterFileSystem() {
1460    return this.fileSystemManager;
1461  }
1462
1463  @Override
1464  public MasterWalManager getMasterWalManager() {
1465    return this.walManager;
1466  }
1467
1468  @Override
1469  public SplitWALManager getSplitWALManager() {
1470    return splitWALManager;
1471  }
1472
1473  @Override
1474  public TableStateManager getTableStateManager() {
1475    return tableStateManager;
1476  }
1477
1478  /*
1479   * Start up all services. If any of these threads gets an unhandled exception then they just die
1480   * with a logged message. This should be fine because in general, we do not expect the master to
1481   * get such unhandled exceptions as OOMEs; it should be lightly loaded. See what HRegionServer
1482   * does if need to install an unexpected exception handler.
1483   */
1484  private void startServiceThreads() throws IOException {
1485    // Start the executor service pools
1486    final int masterOpenRegionPoolSize = conf.getInt(HConstants.MASTER_OPEN_REGION_THREADS,
1487      HConstants.MASTER_OPEN_REGION_THREADS_DEFAULT);
1488    executorService.startExecutorService(executorService.new ExecutorConfig()
1489      .setExecutorType(ExecutorType.MASTER_OPEN_REGION).setCorePoolSize(masterOpenRegionPoolSize));
1490    final int masterCloseRegionPoolSize = conf.getInt(HConstants.MASTER_CLOSE_REGION_THREADS,
1491      HConstants.MASTER_CLOSE_REGION_THREADS_DEFAULT);
1492    executorService.startExecutorService(
1493      executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_CLOSE_REGION)
1494        .setCorePoolSize(masterCloseRegionPoolSize));
1495    final int masterServerOpThreads = conf.getInt(HConstants.MASTER_SERVER_OPERATIONS_THREADS,
1496      HConstants.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT);
1497    executorService.startExecutorService(
1498      executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_SERVER_OPERATIONS)
1499        .setCorePoolSize(masterServerOpThreads));
1500    final int masterServerMetaOpsThreads =
1501      conf.getInt(HConstants.MASTER_META_SERVER_OPERATIONS_THREADS,
1502        HConstants.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT);
1503    executorService.startExecutorService(executorService.new ExecutorConfig()
1504      .setExecutorType(ExecutorType.MASTER_META_SERVER_OPERATIONS)
1505      .setCorePoolSize(masterServerMetaOpsThreads));
1506    final int masterLogReplayThreads = conf.getInt(HConstants.MASTER_LOG_REPLAY_OPS_THREADS,
1507      HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT);
1508    executorService.startExecutorService(executorService.new ExecutorConfig()
1509      .setExecutorType(ExecutorType.M_LOG_REPLAY_OPS).setCorePoolSize(masterLogReplayThreads));
1510    final int masterSnapshotThreads = conf.getInt(SnapshotManager.SNAPSHOT_POOL_THREADS_KEY,
1511      SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT);
1512    executorService.startExecutorService(
1513      executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_SNAPSHOT_OPERATIONS)
1514        .setCorePoolSize(masterSnapshotThreads).setAllowCoreThreadTimeout(true));
1515    final int masterMergeDispatchThreads = conf.getInt(HConstants.MASTER_MERGE_DISPATCH_THREADS,
1516      HConstants.MASTER_MERGE_DISPATCH_THREADS_DEFAULT);
1517    executorService.startExecutorService(
1518      executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_MERGE_OPERATIONS)
1519        .setCorePoolSize(masterMergeDispatchThreads).setAllowCoreThreadTimeout(true));
1520
1521    // We depend on there being only one instance of this executor running
1522    // at a time. To do concurrency, would need fencing of enable/disable of
1523    // tables.
1524    // Any time changing this maxThreads to > 1, pls see the comment at
1525    // AccessController#postCompletedCreateTableAction
1526    executorService.startExecutorService(executorService.new ExecutorConfig()
1527      .setExecutorType(ExecutorType.MASTER_TABLE_OPERATIONS).setCorePoolSize(1));
1528    startProcedureExecutor();
1529
1530    // Create log cleaner thread pool
1531    logCleanerPool = DirScanPool.getLogCleanerScanPool(conf);
1532    Map<String, Object> params = new HashMap<>();
1533    params.put(MASTER, this);
1534    // Start log cleaner thread
1535    int cleanerInterval =
1536      conf.getInt(HBASE_MASTER_CLEANER_INTERVAL, DEFAULT_HBASE_MASTER_CLEANER_INTERVAL);
1537    this.logCleaner =
1538      new LogCleaner(cleanerInterval, this, conf, getMasterWalManager().getFileSystem(),
1539        getMasterWalManager().getOldLogDir(), logCleanerPool, params);
1540    getChoreService().scheduleChore(logCleaner);
1541
1542    // start the hfile archive cleaner thread
1543    Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
1544    // Create archive cleaner thread pool
1545    hfileCleanerPool = DirScanPool.getHFileCleanerScanPool(conf);
1546    this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf,
1547      getMasterFileSystem().getFileSystem(), archiveDir, hfileCleanerPool, params);
1548    getChoreService().scheduleChore(hfileCleaner);
1549
1550    // Regions Reopen based on very high storeFileRefCount is considered enabled
1551    // only if hbase.regions.recovery.store.file.ref.count has value > 0
1552    final int maxStoreFileRefCount = conf.getInt(HConstants.STORE_FILE_REF_COUNT_THRESHOLD,
1553      HConstants.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD);
1554    if (maxStoreFileRefCount > 0) {
1555      this.regionsRecoveryChore = new RegionsRecoveryChore(this, conf, this);
1556      getChoreService().scheduleChore(this.regionsRecoveryChore);
1557    } else {
1558      LOG.info(
1559        "Reopening regions with very high storeFileRefCount is disabled. "
1560          + "Provide threshold value > 0 for {} to enable it.",
1561        HConstants.STORE_FILE_REF_COUNT_THRESHOLD);
1562    }
1563
1564    this.regionsRecoveryConfigManager = new RegionsRecoveryConfigManager(this);
1565
1566    replicationBarrierCleaner =
1567      new ReplicationBarrierCleaner(conf, this, getConnection(), replicationPeerManager);
1568    getChoreService().scheduleChore(replicationBarrierCleaner);
1569
1570    final boolean isSnapshotChoreEnabled = this.snapshotCleanupTracker.isSnapshotCleanupEnabled();
1571    this.snapshotCleanerChore = new SnapshotCleanerChore(this, conf, getSnapshotManager());
1572    if (isSnapshotChoreEnabled) {
1573      getChoreService().scheduleChore(this.snapshotCleanerChore);
1574    } else {
1575      if (LOG.isTraceEnabled()) {
1576        LOG.trace("Snapshot Cleaner Chore is disabled. Not starting up the chore..");
1577      }
1578    }
1579    serviceStarted = true;
1580    if (LOG.isTraceEnabled()) {
1581      LOG.trace("Started service threads");
1582    }
1583  }
1584
1585  protected void stopServiceThreads() {
1586    if (masterJettyServer != null) {
1587      LOG.info("Stopping master jetty server");
1588      try {
1589        masterJettyServer.stop();
1590      } catch (Exception e) {
1591        LOG.error("Failed to stop master jetty server", e);
1592      }
1593    }
1594    stopChoreService();
1595    stopExecutorService();
1596    if (hfileCleanerPool != null) {
1597      hfileCleanerPool.shutdownNow();
1598      hfileCleanerPool = null;
1599    }
1600    if (logCleanerPool != null) {
1601      logCleanerPool.shutdownNow();
1602      logCleanerPool = null;
1603    }
1604    if (maintenanceRegionServer != null) {
1605      maintenanceRegionServer.getRegionServer().stop(HBASE_MASTER_CLEANER_INTERVAL);
1606    }
1607
1608    LOG.debug("Stopping service threads");
1609    // stop procedure executor prior to other services such as server manager and assignment
1610    // manager, as these services are important for some running procedures. See HBASE-24117 for
1611    // example.
1612    stopProcedureExecutor();
1613
1614    if (regionNormalizerManager != null) {
1615      regionNormalizerManager.stop();
1616    }
1617    if (this.quotaManager != null) {
1618      this.quotaManager.stop();
1619    }
1620
1621    if (this.activeMasterManager != null) {
1622      this.activeMasterManager.stop();
1623    }
1624    if (this.serverManager != null) {
1625      this.serverManager.stop();
1626    }
1627    if (this.assignmentManager != null) {
1628      this.assignmentManager.stop();
1629    }
1630
1631    if (masterRegion != null) {
1632      masterRegion.close(isAborted());
1633    }
1634    if (this.walManager != null) {
1635      this.walManager.stop();
1636    }
1637    if (this.fileSystemManager != null) {
1638      this.fileSystemManager.stop();
1639    }
1640    if (this.mpmHost != null) {
1641      this.mpmHost.stop("server shutting down.");
1642    }
1643    if (this.regionServerTracker != null) {
1644      this.regionServerTracker.stop();
1645    }
1646  }
1647
1648  private void createProcedureExecutor() throws IOException {
1649    MasterProcedureEnv procEnv = new MasterProcedureEnv(this);
1650    procedureStore = new RegionProcedureStore(this, masterRegion,
1651      new MasterProcedureEnv.FsUtilsLeaseRecovery(this));
1652    procedureStore.registerListener(new ProcedureStoreListener() {
1653
1654      @Override
1655      public void abortProcess() {
1656        abort("The Procedure Store lost the lease", null);
1657      }
1658    });
1659    MasterProcedureScheduler procedureScheduler = procEnv.getProcedureScheduler();
1660    procedureExecutor = new ProcedureExecutor<>(conf, procEnv, procedureStore, procedureScheduler);
1661    configurationManager.registerObserver(procEnv);
1662
1663    int cpus = Runtime.getRuntime().availableProcessors();
1664    final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, Math.max(
1665      (cpus > 0 ? cpus / 4 : 0), MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
1666    final boolean abortOnCorruption =
1667      conf.getBoolean(MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
1668        MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
1669    procedureStore.start(numThreads);
1670    // Just initialize it but do not start the workers, we will start the workers later by calling
1671    // startProcedureExecutor. See the javadoc for finishActiveMasterInitialization for more
1672    // details.
1673    procedureExecutor.init(numThreads, abortOnCorruption);
1674    if (!procEnv.getRemoteDispatcher().start()) {
1675      throw new HBaseIOException("Failed start of remote dispatcher");
1676    }
1677  }
1678
1679  // will be override in UT
1680  protected void startProcedureExecutor() throws IOException {
1681    procedureExecutor.startWorkers();
1682  }
1683
1684  /**
1685   * Turn on/off Snapshot Cleanup Chore
1686   * @param on indicates whether Snapshot Cleanup Chore is to be run
1687   */
1688  void switchSnapshotCleanup(final boolean on, final boolean synchronous) {
1689    if (synchronous) {
1690      synchronized (this.snapshotCleanerChore) {
1691        switchSnapshotCleanup(on);
1692      }
1693    } else {
1694      switchSnapshotCleanup(on);
1695    }
1696  }
1697
1698  private void switchSnapshotCleanup(final boolean on) {
1699    try {
1700      snapshotCleanupTracker.setSnapshotCleanupEnabled(on);
1701      if (on) {
1702        getChoreService().scheduleChore(this.snapshotCleanerChore);
1703      } else {
1704        this.snapshotCleanerChore.cancel();
1705      }
1706    } catch (KeeperException e) {
1707      LOG.error("Error updating snapshot cleanup mode to {}", on, e);
1708    }
1709  }
1710
1711  private void stopProcedureExecutor() {
1712    if (procedureExecutor != null) {
1713      configurationManager.deregisterObserver(procedureExecutor.getEnvironment());
1714      procedureExecutor.getEnvironment().getRemoteDispatcher().stop();
1715      procedureExecutor.stop();
1716      procedureExecutor.join();
1717      procedureExecutor = null;
1718    }
1719
1720    if (procedureStore != null) {
1721      procedureStore.stop(isAborted());
1722      procedureStore = null;
1723    }
1724  }
1725
1726  protected void stopChores() {
1727    shutdownChore(mobFileCleanerChore);
1728    shutdownChore(mobFileCompactionChore);
1729    shutdownChore(balancerChore);
1730    if (regionNormalizerManager != null) {
1731      shutdownChore(regionNormalizerManager.getRegionNormalizerChore());
1732    }
1733    shutdownChore(clusterStatusChore);
1734    shutdownChore(catalogJanitorChore);
1735    shutdownChore(clusterStatusPublisherChore);
1736    shutdownChore(snapshotQuotaChore);
1737    shutdownChore(logCleaner);
1738    shutdownChore(hfileCleaner);
1739    shutdownChore(replicationBarrierCleaner);
1740    shutdownChore(snapshotCleanerChore);
1741    shutdownChore(hbckChore);
1742    shutdownChore(regionsRecoveryChore);
1743    shutdownChore(rollingUpgradeChore);
1744  }
1745
1746  /**
1747   * @return Get remote side's InetAddress
1748   */
1749  InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
1750    throws UnknownHostException {
1751    // Do it out here in its own little method so can fake an address when
1752    // mocking up in tests.
1753    InetAddress ia = RpcServer.getRemoteIp();
1754
1755    // The call could be from the local regionserver,
1756    // in which case, there is no remote address.
1757    if (ia == null && serverStartCode == startcode) {
1758      InetSocketAddress isa = rpcServices.getSocketAddress();
1759      if (isa != null && isa.getPort() == port) {
1760        ia = isa.getAddress();
1761      }
1762    }
1763    return ia;
1764  }
1765
1766  /**
1767   * @return Maximum time we should run balancer for
1768   */
1769  private int getMaxBalancingTime() {
1770    // if max balancing time isn't set, defaulting it to period time
1771    int maxBalancingTime =
1772      getConfiguration().getInt(HConstants.HBASE_BALANCER_MAX_BALANCING, getConfiguration()
1773        .getInt(HConstants.HBASE_BALANCER_PERIOD, HConstants.DEFAULT_HBASE_BALANCER_PERIOD));
1774    return maxBalancingTime;
1775  }
1776
1777  /**
1778   * @return Maximum number of regions in transition
1779   */
1780  private int getMaxRegionsInTransition() {
1781    int numRegions = this.assignmentManager.getRegionStates().getRegionAssignments().size();
1782    return Math.max((int) Math.floor(numRegions * this.maxRitPercent), 1);
1783  }
1784
1785  /**
1786   * It first sleep to the next balance plan start time. Meanwhile, throttling by the max number
1787   * regions in transition to protect availability.
1788   * @param nextBalanceStartTime   The next balance plan start time
1789   * @param maxRegionsInTransition max number of regions in transition
1790   * @param cutoffTime             when to exit balancer
1791   */
1792  private void balanceThrottling(long nextBalanceStartTime, int maxRegionsInTransition,
1793    long cutoffTime) {
1794    boolean interrupted = false;
1795
1796    // Sleep to next balance plan start time
1797    // But if there are zero regions in transition, it can skip sleep to speed up.
1798    while (
1799      !interrupted && EnvironmentEdgeManager.currentTime() < nextBalanceStartTime
1800        && this.assignmentManager.getRegionStates().hasRegionsInTransition()
1801    ) {
1802      try {
1803        Thread.sleep(100);
1804      } catch (InterruptedException ie) {
1805        interrupted = true;
1806      }
1807    }
1808
1809    // Throttling by max number regions in transition
1810    while (
1811      !interrupted && maxRegionsInTransition > 0
1812        && this.assignmentManager.getRegionStates().getRegionsInTransitionCount()
1813            >= maxRegionsInTransition
1814        && EnvironmentEdgeManager.currentTime() <= cutoffTime
1815    ) {
1816      try {
1817        // sleep if the number of regions in transition exceeds the limit
1818        Thread.sleep(100);
1819      } catch (InterruptedException ie) {
1820        interrupted = true;
1821      }
1822    }
1823
1824    if (interrupted) Thread.currentThread().interrupt();
1825  }
1826
1827  public BalanceResponse balance() throws IOException {
1828    return balance(BalanceRequest.defaultInstance());
1829  }
1830
1831  /**
1832   * Trigger a normal balance, see {@link HMaster#balance()} . If the balance is not executed this
1833   * time, the metrics related to the balance will be updated. When balance is running, related
1834   * metrics will be updated at the same time. But if some checking logic failed and cause the
1835   * balancer exit early, we lost the chance to update balancer metrics. This will lead to user
1836   * missing the latest balancer info.
1837   */
1838  public BalanceResponse balanceOrUpdateMetrics() throws IOException {
1839    synchronized (this.balancer) {
1840      BalanceResponse response = balance();
1841      if (!response.isBalancerRan()) {
1842        Map<TableName, Map<ServerName, List<RegionInfo>>> assignments =
1843          this.assignmentManager.getRegionStates().getAssignmentsForBalancer(this.tableStateManager,
1844            this.serverManager.getOnlineServersList());
1845        for (Map<ServerName, List<RegionInfo>> serverMap : assignments.values()) {
1846          serverMap.keySet().removeAll(this.serverManager.getDrainingServersList());
1847        }
1848        this.balancer.updateBalancerLoadInfo(assignments);
1849      }
1850      return response;
1851    }
1852  }
1853
1854  /**
1855   * Checks master state before initiating action over region topology.
1856   * @param action the name of the action under consideration, for logging.
1857   * @return {@code true} when the caller should exit early, {@code false} otherwise.
1858   */
1859  @Override
1860  public boolean skipRegionManagementAction(final String action) {
1861    // Note: this method could be `default` on MasterServices if but for logging.
1862    if (!isInitialized()) {
1863      LOG.debug("Master has not been initialized, don't run {}.", action);
1864      return true;
1865    }
1866    if (this.getServerManager().isClusterShutdown()) {
1867      LOG.info("Cluster is shutting down, don't run {}.", action);
1868      return true;
1869    }
1870    if (isInMaintenanceMode()) {
1871      LOG.info("Master is in maintenance mode, don't run {}.", action);
1872      return true;
1873    }
1874    return false;
1875  }
1876
1877  public BalanceResponse balance(BalanceRequest request) throws IOException {
1878    checkInitialized();
1879
1880    BalanceResponse.Builder responseBuilder = BalanceResponse.newBuilder();
1881
1882    if (
1883      loadBalancerTracker == null || !(loadBalancerTracker.isBalancerOn() || request.isDryRun())
1884    ) {
1885      return responseBuilder.build();
1886    }
1887
1888    if (skipRegionManagementAction("balancer")) {
1889      return responseBuilder.build();
1890    }
1891
1892    synchronized (this.balancer) {
1893      // Only allow one balance run at at time.
1894      if (this.assignmentManager.hasRegionsInTransition()) {
1895        List<RegionStateNode> regionsInTransition = assignmentManager.getRegionsInTransition();
1896        // if hbase:meta region is in transition, result of assignment cannot be recorded
1897        // ignore the force flag in that case
1898        boolean metaInTransition = assignmentManager.isMetaRegionInTransition();
1899        List<RegionStateNode> toPrint = regionsInTransition;
1900        int max = 5;
1901        boolean truncated = false;
1902        if (regionsInTransition.size() > max) {
1903          toPrint = regionsInTransition.subList(0, max);
1904          truncated = true;
1905        }
1906
1907        if (!request.isIgnoreRegionsInTransition() || metaInTransition) {
1908          LOG.info("Not running balancer (ignoreRIT=false" + ", metaRIT=" + metaInTransition
1909            + ") because " + regionsInTransition.size() + " region(s) in transition: " + toPrint
1910            + (truncated ? "(truncated list)" : ""));
1911          return responseBuilder.build();
1912        }
1913      }
1914      if (this.serverManager.areDeadServersInProgress()) {
1915        LOG.info("Not running balancer because processing dead regionserver(s): "
1916          + this.serverManager.getDeadServers());
1917        return responseBuilder.build();
1918      }
1919
1920      if (this.cpHost != null) {
1921        try {
1922          if (this.cpHost.preBalance(request)) {
1923            LOG.debug("Coprocessor bypassing balancer request");
1924            return responseBuilder.build();
1925          }
1926        } catch (IOException ioe) {
1927          LOG.error("Error invoking master coprocessor preBalance()", ioe);
1928          return responseBuilder.build();
1929        }
1930      }
1931
1932      Map<TableName, Map<ServerName, List<RegionInfo>>> assignments =
1933        this.assignmentManager.getRegionStates().getAssignmentsForBalancer(tableStateManager,
1934          this.serverManager.getOnlineServersList());
1935      for (Map<ServerName, List<RegionInfo>> serverMap : assignments.values()) {
1936        serverMap.keySet().removeAll(this.serverManager.getDrainingServersList());
1937      }
1938
1939      // Give the balancer the current cluster state.
1940      this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1941
1942      List<RegionPlan> plans = this.balancer.balanceCluster(assignments);
1943
1944      responseBuilder.setBalancerRan(true).setMovesCalculated(plans == null ? 0 : plans.size());
1945
1946      if (skipRegionManagementAction("balancer")) {
1947        // make one last check that the cluster isn't shutting down before proceeding.
1948        return responseBuilder.build();
1949      }
1950
1951      // For dry run we don't actually want to execute the moves, but we do want
1952      // to execute the coprocessor below
1953      List<RegionPlan> sucRPs =
1954        request.isDryRun() ? Collections.emptyList() : executeRegionPlansWithThrottling(plans);
1955
1956      if (this.cpHost != null) {
1957        try {
1958          this.cpHost.postBalance(request, sucRPs);
1959        } catch (IOException ioe) {
1960          // balancing already succeeded so don't change the result
1961          LOG.error("Error invoking master coprocessor postBalance()", ioe);
1962        }
1963      }
1964
1965      responseBuilder.setMovesExecuted(sucRPs.size());
1966    }
1967
1968    // If LoadBalancer did not generate any plans, it means the cluster is already balanced.
1969    // Return true indicating a success.
1970    return responseBuilder.build();
1971  }
1972
1973  /**
1974   * Execute region plans with throttling
1975   * @param plans to execute
1976   * @return succeeded plans
1977   */
1978  public List<RegionPlan> executeRegionPlansWithThrottling(List<RegionPlan> plans) {
1979    List<RegionPlan> successRegionPlans = new ArrayList<>();
1980    int maxRegionsInTransition = getMaxRegionsInTransition();
1981    long balanceStartTime = EnvironmentEdgeManager.currentTime();
1982    long cutoffTime = balanceStartTime + this.maxBalancingTime;
1983    int rpCount = 0; // number of RegionPlans balanced so far
1984    if (plans != null && !plans.isEmpty()) {
1985      int balanceInterval = this.maxBalancingTime / plans.size();
1986      LOG.info(
1987        "Balancer plans size is " + plans.size() + ", the balance interval is " + balanceInterval
1988          + " ms, and the max number regions in transition is " + maxRegionsInTransition);
1989
1990      for (RegionPlan plan : plans) {
1991        LOG.info("balance " + plan);
1992        // TODO: bulk assign
1993        try {
1994          this.assignmentManager.balance(plan);
1995        } catch (HBaseIOException hioe) {
1996          // should ignore failed plans here, avoiding the whole balance plans be aborted
1997          // later calls of balance() can fetch up the failed and skipped plans
1998          LOG.warn("Failed balance plan {}, skipping...", plan, hioe);
1999        }
2000        // rpCount records balance plans processed, does not care if a plan succeeds
2001        rpCount++;
2002        successRegionPlans.add(plan);
2003
2004        if (this.maxBalancingTime > 0) {
2005          balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,
2006            cutoffTime);
2007        }
2008
2009        // if performing next balance exceeds cutoff time, exit the loop
2010        if (
2011          this.maxBalancingTime > 0 && rpCount < plans.size()
2012            && EnvironmentEdgeManager.currentTime() > cutoffTime
2013        ) {
2014          // TODO: After balance, there should not be a cutoff time (keeping it as
2015          // a security net for now)
2016          LOG.debug(
2017            "No more balancing till next balance run; maxBalanceTime=" + this.maxBalancingTime);
2018          break;
2019        }
2020      }
2021    }
2022    LOG.debug("Balancer is going into sleep until next period in {}ms", getConfiguration()
2023      .getInt(HConstants.HBASE_BALANCER_PERIOD, HConstants.DEFAULT_HBASE_BALANCER_PERIOD));
2024    return successRegionPlans;
2025  }
2026
2027  @Override
2028  public RegionNormalizerManager getRegionNormalizerManager() {
2029    return regionNormalizerManager;
2030  }
2031
2032  @Override
2033  public boolean normalizeRegions(final NormalizeTableFilterParams ntfp,
2034    final boolean isHighPriority) throws IOException {
2035    if (regionNormalizerManager == null || !regionNormalizerManager.isNormalizerOn()) {
2036      LOG.debug("Region normalization is disabled, don't run region normalizer.");
2037      return false;
2038    }
2039    if (skipRegionManagementAction("region normalizer")) {
2040      return false;
2041    }
2042    if (assignmentManager.hasRegionsInTransition()) {
2043      return false;
2044    }
2045
2046    final Set<TableName> matchingTables = getTableDescriptors(new LinkedList<>(),
2047      ntfp.getNamespace(), ntfp.getRegex(), ntfp.getTableNames(), false).stream()
2048        .map(TableDescriptor::getTableName).collect(Collectors.toSet());
2049    final Set<TableName> allEnabledTables =
2050      tableStateManager.getTablesInStates(TableState.State.ENABLED);
2051    final List<TableName> targetTables =
2052      new ArrayList<>(Sets.intersection(matchingTables, allEnabledTables));
2053    Collections.shuffle(targetTables);
2054    return regionNormalizerManager.normalizeRegions(targetTables, isHighPriority);
2055  }
2056
2057  /**
2058   * @return Client info for use as prefix on an audit log string; who did an action
2059   */
2060  @Override
2061  public String getClientIdAuditPrefix() {
2062    return "Client=" + RpcServer.getRequestUserName().orElse(null) + "/"
2063      + RpcServer.getRemoteAddress().orElse(null);
2064  }
2065
2066  /**
2067   * Switch for the background CatalogJanitor thread. Used for testing. The thread will continue to
2068   * run. It will just be a noop if disabled.
2069   * @param b If false, the catalog janitor won't do anything.
2070   */
2071  public void setCatalogJanitorEnabled(final boolean b) {
2072    this.catalogJanitorChore.setEnabled(b);
2073  }
2074
2075  @Override
2076  public long mergeRegions(final RegionInfo[] regionsToMerge, final boolean forcible, final long ng,
2077    final long nonce) throws IOException {
2078    checkInitialized();
2079
2080    if (!isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
2081      String regionsStr = Arrays.deepToString(regionsToMerge);
2082      LOG.warn("Merge switch is off! skip merge of " + regionsStr);
2083      throw new DoNotRetryIOException(
2084        "Merge of " + regionsStr + " failed because merge switch is off");
2085    }
2086
2087    final String mergeRegionsStr = Arrays.stream(regionsToMerge).map(RegionInfo::getEncodedName)
2088      .collect(Collectors.joining(", "));
2089    return MasterProcedureUtil.submitProcedure(new NonceProcedureRunnable(this, ng, nonce) {
2090      @Override
2091      protected void run() throws IOException {
2092        getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge);
2093        String aid = getClientIdAuditPrefix();
2094        LOG.info("{} merge regions {}", aid, mergeRegionsStr);
2095        submitProcedure(new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(),
2096          regionsToMerge, forcible));
2097        getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge);
2098      }
2099
2100      @Override
2101      protected String getDescription() {
2102        return "MergeTableProcedure";
2103      }
2104    });
2105  }
2106
2107  @Override
2108  public long splitRegion(final RegionInfo regionInfo, final byte[] splitRow, final long nonceGroup,
2109    final long nonce) throws IOException {
2110    checkInitialized();
2111
2112    if (!isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
2113      LOG.warn("Split switch is off! skip split of " + regionInfo);
2114      throw new DoNotRetryIOException(
2115        "Split region " + regionInfo.getRegionNameAsString() + " failed due to split switch off");
2116    }
2117
2118    return MasterProcedureUtil
2119      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2120        @Override
2121        protected void run() throws IOException {
2122          getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo.getTable(), splitRow);
2123          LOG.info(getClientIdAuditPrefix() + " split " + regionInfo.getRegionNameAsString());
2124
2125          // Execute the operation asynchronously
2126          submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo, splitRow));
2127        }
2128
2129        @Override
2130        protected String getDescription() {
2131          return "SplitTableProcedure";
2132        }
2133      });
2134  }
2135
2136  private void warmUpRegion(ServerName server, RegionInfo region) {
2137    FutureUtils.addListener(asyncClusterConnection.getRegionServerAdmin(server)
2138      .warmupRegion(RequestConverter.buildWarmupRegionRequest(region)), (r, e) -> {
2139        if (e != null) {
2140          LOG.warn("Failed to warm up region {} on server {}", region, server, e);
2141        }
2142      });
2143  }
2144
2145  // Public so can be accessed by tests. Blocks until move is done.
2146  // Replace with an async implementation from which you can get
2147  // a success/failure result.
2148  @InterfaceAudience.Private
2149  public void move(final byte[] encodedRegionName, byte[] destServerName) throws IOException {
2150    RegionState regionState =
2151      assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));
2152
2153    RegionInfo hri;
2154    if (regionState != null) {
2155      hri = regionState.getRegion();
2156    } else {
2157      throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
2158    }
2159
2160    ServerName dest;
2161    List<ServerName> exclude = hri.getTable().isSystemTable()
2162      ? assignmentManager.getExcludedServersForSystemTable()
2163      : new ArrayList<>(1);
2164    if (
2165      destServerName != null && exclude.contains(ServerName.valueOf(Bytes.toString(destServerName)))
2166    ) {
2167      LOG.info(Bytes.toString(encodedRegionName) + " can not move to "
2168        + Bytes.toString(destServerName) + " because the server is in exclude list");
2169      destServerName = null;
2170    }
2171    if (destServerName == null || destServerName.length == 0) {
2172      LOG.info("Passed destination servername is null/empty so " + "choosing a server at random");
2173      exclude.add(regionState.getServerName());
2174      final List<ServerName> destServers = this.serverManager.createDestinationServersList(exclude);
2175      dest = balancer.randomAssignment(hri, destServers);
2176      if (dest == null) {
2177        LOG.debug("Unable to determine a plan to assign " + hri);
2178        return;
2179      }
2180    } else {
2181      ServerName candidate = ServerName.valueOf(Bytes.toString(destServerName));
2182      dest = balancer.randomAssignment(hri, Lists.newArrayList(candidate));
2183      if (dest == null) {
2184        LOG.debug("Unable to determine a plan to assign " + hri);
2185        return;
2186      }
2187      // TODO: deal with table on master for rs group.
2188      if (dest.equals(serverName)) {
2189        // To avoid unnecessary region moving later by balancer. Don't put user
2190        // regions on master.
2191        LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
2192          + " to avoid unnecessary region moving later by load balancer,"
2193          + " because it should not be on master");
2194        return;
2195      }
2196    }
2197
2198    if (dest.equals(regionState.getServerName())) {
2199      LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
2200        + " because region already assigned to the same server " + dest + ".");
2201      return;
2202    }
2203
2204    // Now we can do the move
2205    RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);
2206    assert rp.getDestination() != null : rp.toString() + " " + dest;
2207
2208    try {
2209      checkInitialized();
2210      if (this.cpHost != null) {
2211        this.cpHost.preMove(hri, rp.getSource(), rp.getDestination());
2212      }
2213
2214      TransitRegionStateProcedure proc =
2215        this.assignmentManager.createMoveRegionProcedure(rp.getRegionInfo(), rp.getDestination());
2216      if (conf.getBoolean(WARMUP_BEFORE_MOVE, DEFAULT_WARMUP_BEFORE_MOVE)) {
2217        // Warmup the region on the destination before initiating the move.
2218        // A region server could reject the close request because it either does not
2219        // have the specified region or the region is being split.
2220        LOG.info(getClientIdAuditPrefix() + " move " + rp + ", warming up region on "
2221          + rp.getDestination());
2222        warmUpRegion(rp.getDestination(), hri);
2223      }
2224      LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
2225      Future<byte[]> future = ProcedureSyncWait.submitProcedure(this.procedureExecutor, proc);
2226      try {
2227        // Is this going to work? Will we throw exception on error?
2228        // TODO: CompletableFuture rather than this stunted Future.
2229        future.get();
2230      } catch (InterruptedException | ExecutionException e) {
2231        throw new HBaseIOException(e);
2232      }
2233      if (this.cpHost != null) {
2234        this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
2235      }
2236    } catch (IOException ioe) {
2237      if (ioe instanceof HBaseIOException) {
2238        throw (HBaseIOException) ioe;
2239      }
2240      throw new HBaseIOException(ioe);
2241    }
2242  }
2243
2244  @Override
2245  public long createTable(final TableDescriptor tableDescriptor, final byte[][] splitKeys,
2246    final long nonceGroup, final long nonce) throws IOException {
2247    checkInitialized();
2248    TableDescriptor desc = getMasterCoprocessorHost().preCreateTableRegionsInfos(tableDescriptor);
2249    if (desc == null) {
2250      throw new IOException("Creation for " + tableDescriptor + " is canceled by CP");
2251    }
2252    String namespace = desc.getTableName().getNamespaceAsString();
2253    this.clusterSchemaService.getNamespace(namespace);
2254
2255    RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(desc, splitKeys);
2256    TableDescriptorChecker.sanityCheck(conf, desc);
2257
2258    return MasterProcedureUtil
2259      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2260        @Override
2261        protected void run() throws IOException {
2262          getMaster().getMasterCoprocessorHost().preCreateTable(desc, newRegions);
2263
2264          LOG.info(getClientIdAuditPrefix() + " create " + desc);
2265
2266          // TODO: We can handle/merge duplicate requests, and differentiate the case of
2267          // TableExistsException by saying if the schema is the same or not.
2268          //
2269          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2270          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2271          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2272          submitProcedure(
2273            new CreateTableProcedure(procedureExecutor.getEnvironment(), desc, newRegions, latch));
2274          latch.await();
2275
2276          getMaster().getMasterCoprocessorHost().postCreateTable(desc, newRegions);
2277        }
2278
2279        @Override
2280        protected String getDescription() {
2281          return "CreateTableProcedure";
2282        }
2283      });
2284  }
2285
2286  @Override
2287  public long createSystemTable(final TableDescriptor tableDescriptor) throws IOException {
2288    if (isStopped()) {
2289      throw new MasterNotRunningException();
2290    }
2291
2292    TableName tableName = tableDescriptor.getTableName();
2293    if (!(tableName.isSystemTable())) {
2294      throw new IllegalArgumentException(
2295        "Only system table creation can use this createSystemTable API");
2296    }
2297
2298    RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, null);
2299
2300    LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
2301
2302    // This special create table is called locally to master. Therefore, no RPC means no need
2303    // to use nonce to detect duplicated RPC call.
2304    long procId = this.procedureExecutor.submitProcedure(
2305      new CreateTableProcedure(procedureExecutor.getEnvironment(), tableDescriptor, newRegions));
2306
2307    return procId;
2308  }
2309
2310  private void startActiveMasterManager(int infoPort) throws KeeperException {
2311    String backupZNode = ZNodePaths.joinZNode(zooKeeper.getZNodePaths().backupMasterAddressesZNode,
2312      serverName.toString());
2313    /*
2314     * Add a ZNode for ourselves in the backup master directory since we may not become the active
2315     * master. If so, we want the actual active master to know we are backup masters, so that it
2316     * won't assign regions to us if so configured. If we become the active master later,
2317     * ActiveMasterManager will delete this node explicitly. If we crash before then, ZooKeeper will
2318     * delete this node for us since it is ephemeral.
2319     */
2320    LOG.info("Adding backup master ZNode " + backupZNode);
2321    if (!MasterAddressTracker.setMasterAddress(zooKeeper, backupZNode, serverName, infoPort)) {
2322      LOG.warn("Failed create of " + backupZNode + " by " + serverName);
2323    }
2324    this.activeMasterManager.setInfoPort(infoPort);
2325    int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
2326    // If we're a backup master, stall until a primary to write this address
2327    if (conf.getBoolean(HConstants.MASTER_TYPE_BACKUP, HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
2328      LOG.debug("HMaster started in backup mode. Stalling until master znode is written.");
2329      // This will only be a minute or so while the cluster starts up,
2330      // so don't worry about setting watches on the parent znode
2331      while (!activeMasterManager.hasActiveMaster()) {
2332        LOG.debug("Waiting for master address and cluster state znode to be written.");
2333        Threads.sleep(timeout);
2334      }
2335    }
2336    MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
2337    status.setDescription("Master startup");
2338    try {
2339      if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
2340        finishActiveMasterInitialization(status);
2341      }
2342    } catch (Throwable t) {
2343      status.setStatus("Failed to become active: " + t.getMessage());
2344      LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
2345      // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
2346      if (
2347        t instanceof NoClassDefFoundError
2348          && t.getMessage().contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")
2349      ) {
2350        // improved error message for this special case
2351        abort("HBase is having a problem with its Hadoop jars.  You may need to recompile "
2352          + "HBase against Hadoop version " + org.apache.hadoop.util.VersionInfo.getVersion()
2353          + " or change your hadoop jars to start properly", t);
2354      } else {
2355        abort("Unhandled exception. Starting shutdown.", t);
2356      }
2357    } finally {
2358      status.cleanup();
2359    }
2360  }
2361
2362  private static boolean isCatalogTable(final TableName tableName) {
2363    return tableName.equals(TableName.META_TABLE_NAME);
2364  }
2365
2366  @Override
2367  public long deleteTable(final TableName tableName, final long nonceGroup, final long nonce)
2368    throws IOException {
2369    checkInitialized();
2370
2371    return MasterProcedureUtil
2372      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2373        @Override
2374        protected void run() throws IOException {
2375          getMaster().getMasterCoprocessorHost().preDeleteTable(tableName);
2376
2377          LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
2378
2379          // TODO: We can handle/merge duplicate request
2380          //
2381          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2382          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2383          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2384          submitProcedure(
2385            new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch));
2386          latch.await();
2387
2388          getMaster().getMasterCoprocessorHost().postDeleteTable(tableName);
2389        }
2390
2391        @Override
2392        protected String getDescription() {
2393          return "DeleteTableProcedure";
2394        }
2395      });
2396  }
2397
2398  @Override
2399  public long truncateTable(final TableName tableName, final boolean preserveSplits,
2400    final long nonceGroup, final long nonce) throws IOException {
2401    checkInitialized();
2402
2403    return MasterProcedureUtil
2404      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2405        @Override
2406        protected void run() throws IOException {
2407          getMaster().getMasterCoprocessorHost().preTruncateTable(tableName);
2408
2409          LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
2410          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
2411          submitProcedure(new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName,
2412            preserveSplits, latch));
2413          latch.await();
2414
2415          getMaster().getMasterCoprocessorHost().postTruncateTable(tableName);
2416        }
2417
2418        @Override
2419        protected String getDescription() {
2420          return "TruncateTableProcedure";
2421        }
2422      });
2423  }
2424
2425  @Override
2426  public long addColumn(final TableName tableName, final ColumnFamilyDescriptor column,
2427    final long nonceGroup, final long nonce) throws IOException {
2428    checkInitialized();
2429    checkTableExists(tableName);
2430
2431    return modifyTable(tableName, new TableDescriptorGetter() {
2432
2433      @Override
2434      public TableDescriptor get() throws IOException {
2435        TableDescriptor old = getTableDescriptors().get(tableName);
2436        if (old.hasColumnFamily(column.getName())) {
2437          throw new InvalidFamilyOperationException("Column family '" + column.getNameAsString()
2438            + "' in table '" + tableName + "' already exists so cannot be added");
2439        }
2440
2441        return TableDescriptorBuilder.newBuilder(old).setColumnFamily(column).build();
2442      }
2443    }, nonceGroup, nonce, true);
2444  }
2445
2446  /**
2447   * Implement to return TableDescriptor after pre-checks
2448   */
2449  protected interface TableDescriptorGetter {
2450    TableDescriptor get() throws IOException;
2451  }
2452
2453  @Override
2454  public long modifyColumn(final TableName tableName, final ColumnFamilyDescriptor descriptor,
2455    final long nonceGroup, final long nonce) throws IOException {
2456    checkInitialized();
2457    checkTableExists(tableName);
2458    return modifyTable(tableName, new TableDescriptorGetter() {
2459
2460      @Override
2461      public TableDescriptor get() throws IOException {
2462        TableDescriptor old = getTableDescriptors().get(tableName);
2463        if (!old.hasColumnFamily(descriptor.getName())) {
2464          throw new InvalidFamilyOperationException("Family '" + descriptor.getNameAsString()
2465            + "' does not exist, so it cannot be modified");
2466        }
2467
2468        return TableDescriptorBuilder.newBuilder(old).modifyColumnFamily(descriptor).build();
2469      }
2470    }, nonceGroup, nonce, true);
2471  }
2472
2473  @Override
2474  public long modifyColumnStoreFileTracker(TableName tableName, byte[] family, String dstSFT,
2475    long nonceGroup, long nonce) throws IOException {
2476    checkInitialized();
2477    return MasterProcedureUtil
2478      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2479
2480        @Override
2481        protected void run() throws IOException {
2482          String sft = getMaster().getMasterCoprocessorHost()
2483            .preModifyColumnFamilyStoreFileTracker(tableName, family, dstSFT);
2484          LOG.info("{} modify column {} store file tracker of table {} to {}",
2485            getClientIdAuditPrefix(), Bytes.toStringBinary(family), tableName, sft);
2486          submitProcedure(new ModifyColumnFamilyStoreFileTrackerProcedure(
2487            procedureExecutor.getEnvironment(), tableName, family, sft));
2488          getMaster().getMasterCoprocessorHost().postModifyColumnFamilyStoreFileTracker(tableName,
2489            family, dstSFT);
2490        }
2491
2492        @Override
2493        protected String getDescription() {
2494          return "ModifyColumnFamilyStoreFileTrackerProcedure";
2495        }
2496      });
2497  }
2498
2499  @Override
2500  public long deleteColumn(final TableName tableName, final byte[] columnName,
2501    final long nonceGroup, final long nonce) throws IOException {
2502    checkInitialized();
2503    checkTableExists(tableName);
2504
2505    return modifyTable(tableName, new TableDescriptorGetter() {
2506
2507      @Override
2508      public TableDescriptor get() throws IOException {
2509        TableDescriptor old = getTableDescriptors().get(tableName);
2510
2511        if (!old.hasColumnFamily(columnName)) {
2512          throw new InvalidFamilyOperationException(
2513            "Family '" + Bytes.toString(columnName) + "' does not exist, so it cannot be deleted");
2514        }
2515        if (old.getColumnFamilyCount() == 1) {
2516          throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)
2517            + "' is the only column family in the table, so it cannot be deleted");
2518        }
2519        return TableDescriptorBuilder.newBuilder(old).removeColumnFamily(columnName).build();
2520      }
2521    }, nonceGroup, nonce, true);
2522  }
2523
2524  @Override
2525  public long enableTable(final TableName tableName, final long nonceGroup, final long nonce)
2526    throws IOException {
2527    checkInitialized();
2528
2529    return MasterProcedureUtil
2530      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2531        @Override
2532        protected void run() throws IOException {
2533          getMaster().getMasterCoprocessorHost().preEnableTable(tableName);
2534
2535          // Normally, it would make sense for this authorization check to exist inside
2536          // AccessController, but because the authorization check is done based on internal state
2537          // (rather than explicit permissions) we'll do the check here instead of in the
2538          // coprocessor.
2539          MasterQuotaManager quotaManager = getMasterQuotaManager();
2540          if (quotaManager != null) {
2541            if (quotaManager.isQuotaInitialized()) {
2542              SpaceQuotaSnapshot currSnapshotOfTable =
2543                QuotaTableUtil.getCurrentSnapshotFromQuotaTable(getConnection(), tableName);
2544              if (currSnapshotOfTable != null) {
2545                SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();
2546                if (
2547                  quotaStatus.isInViolation()
2548                    && SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy().orElse(null)
2549                ) {
2550                  throw new AccessDeniedException("Enabling the table '" + tableName
2551                    + "' is disallowed due to a violated space quota.");
2552                }
2553              }
2554            } else if (LOG.isTraceEnabled()) {
2555              LOG
2556                .trace("Unable to check for space quotas as the MasterQuotaManager is not enabled");
2557            }
2558          }
2559
2560          LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
2561
2562          // Execute the operation asynchronously - client will check the progress of the operation
2563          // In case the request is from a <1.1 client before returning,
2564          // we want to make sure that the table is prepared to be
2565          // enabled (the table is locked and the table state is set).
2566          // Note: if the procedure throws exception, we will catch it and rethrow.
2567          final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
2568          submitProcedure(
2569            new EnableTableProcedure(procedureExecutor.getEnvironment(), tableName, prepareLatch));
2570          prepareLatch.await();
2571
2572          getMaster().getMasterCoprocessorHost().postEnableTable(tableName);
2573        }
2574
2575        @Override
2576        protected String getDescription() {
2577          return "EnableTableProcedure";
2578        }
2579      });
2580  }
2581
2582  @Override
2583  public long disableTable(final TableName tableName, final long nonceGroup, final long nonce)
2584    throws IOException {
2585    checkInitialized();
2586
2587    return MasterProcedureUtil
2588      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2589        @Override
2590        protected void run() throws IOException {
2591          getMaster().getMasterCoprocessorHost().preDisableTable(tableName);
2592
2593          LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
2594
2595          // Execute the operation asynchronously - client will check the progress of the operation
2596          // In case the request is from a <1.1 client before returning,
2597          // we want to make sure that the table is prepared to be
2598          // enabled (the table is locked and the table state is set).
2599          // Note: if the procedure throws exception, we will catch it and rethrow.
2600          //
2601          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2602          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2603          final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createBlockingLatch();
2604          submitProcedure(new DisableTableProcedure(procedureExecutor.getEnvironment(), tableName,
2605            false, prepareLatch));
2606          prepareLatch.await();
2607
2608          getMaster().getMasterCoprocessorHost().postDisableTable(tableName);
2609        }
2610
2611        @Override
2612        protected String getDescription() {
2613          return "DisableTableProcedure";
2614        }
2615      });
2616  }
2617
2618  private long modifyTable(final TableName tableName,
2619    final TableDescriptorGetter newDescriptorGetter, final long nonceGroup, final long nonce,
2620    final boolean shouldCheckDescriptor) throws IOException {
2621    return MasterProcedureUtil
2622      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2623        @Override
2624        protected void run() throws IOException {
2625          TableDescriptor oldDescriptor = getMaster().getTableDescriptors().get(tableName);
2626          TableDescriptor newDescriptor = getMaster().getMasterCoprocessorHost()
2627            .preModifyTable(tableName, oldDescriptor, newDescriptorGetter.get());
2628          TableDescriptorChecker.sanityCheck(conf, newDescriptor);
2629          LOG.info("{} modify table {} from {} to {}", getClientIdAuditPrefix(), tableName,
2630            oldDescriptor, newDescriptor);
2631
2632          // Execute the operation synchronously - wait for the operation completes before
2633          // continuing.
2634          //
2635          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2636          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2637          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2638          submitProcedure(new ModifyTableProcedure(procedureExecutor.getEnvironment(),
2639            newDescriptor, latch, oldDescriptor, shouldCheckDescriptor));
2640          latch.await();
2641
2642          getMaster().getMasterCoprocessorHost().postModifyTable(tableName, oldDescriptor,
2643            newDescriptor);
2644        }
2645
2646        @Override
2647        protected String getDescription() {
2648          return "ModifyTableProcedure";
2649        }
2650      });
2651
2652  }
2653
2654  @Override
2655  public long modifyTable(final TableName tableName, final TableDescriptor newDescriptor,
2656    final long nonceGroup, final long nonce) throws IOException {
2657    checkInitialized();
2658    return modifyTable(tableName, new TableDescriptorGetter() {
2659      @Override
2660      public TableDescriptor get() throws IOException {
2661        return newDescriptor;
2662      }
2663    }, nonceGroup, nonce, false);
2664
2665  }
2666
2667  @Override
2668  public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup,
2669    long nonce) throws IOException {
2670    checkInitialized();
2671    return MasterProcedureUtil
2672      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2673
2674        @Override
2675        protected void run() throws IOException {
2676          String sft = getMaster().getMasterCoprocessorHost()
2677            .preModifyTableStoreFileTracker(tableName, dstSFT);
2678          LOG.info("{} modify table store file tracker of table {} to {}", getClientIdAuditPrefix(),
2679            tableName, sft);
2680          submitProcedure(new ModifyTableStoreFileTrackerProcedure(
2681            procedureExecutor.getEnvironment(), tableName, sft));
2682          getMaster().getMasterCoprocessorHost().postModifyTableStoreFileTracker(tableName, sft);
2683        }
2684
2685        @Override
2686        protected String getDescription() {
2687          return "ModifyTableStoreFileTrackerProcedure";
2688        }
2689      });
2690  }
2691
2692  public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long nonceGroup,
2693    final long nonce, final boolean restoreAcl, final String customSFT) throws IOException {
2694    checkInitialized();
2695    getSnapshotManager().checkSnapshotSupport();
2696
2697    // Ensure namespace exists. Will throw exception if non-known NS.
2698    final TableName dstTable = TableName.valueOf(snapshotDesc.getTable());
2699    getClusterSchema().getNamespace(dstTable.getNamespaceAsString());
2700
2701    return MasterProcedureUtil
2702      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2703        @Override
2704        protected void run() throws IOException {
2705          setProcId(getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(),
2706            restoreAcl, customSFT));
2707        }
2708
2709        @Override
2710        protected String getDescription() {
2711          return "RestoreSnapshotProcedure";
2712        }
2713      });
2714  }
2715
2716  private void checkTableExists(final TableName tableName)
2717    throws IOException, TableNotFoundException {
2718    if (!tableDescriptors.exists(tableName)) {
2719      throw new TableNotFoundException(tableName);
2720    }
2721  }
2722
2723  @Override
2724  public void checkTableModifiable(final TableName tableName)
2725    throws IOException, TableNotFoundException, TableNotDisabledException {
2726    if (isCatalogTable(tableName)) {
2727      throw new IOException("Can't modify catalog tables");
2728    }
2729    checkTableExists(tableName);
2730    TableState ts = getTableStateManager().getTableState(tableName);
2731    if (!ts.isDisabled()) {
2732      throw new TableNotDisabledException("Not DISABLED; " + ts);
2733    }
2734  }
2735
2736  public ClusterMetrics getClusterMetricsWithoutCoprocessor() throws InterruptedIOException {
2737    return getClusterMetricsWithoutCoprocessor(EnumSet.allOf(Option.class));
2738  }
2739
2740  public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet<Option> options)
2741    throws InterruptedIOException {
2742    ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
2743    // given that hbase1 can't submit the request with Option,
2744    // we return all information to client if the list of Option is empty.
2745    if (options.isEmpty()) {
2746      options = EnumSet.allOf(Option.class);
2747    }
2748
2749    // TASKS and/or LIVE_SERVERS will populate this map, which will be given to the builder if
2750    // not null after option processing completes.
2751    Map<ServerName, ServerMetrics> serverMetricsMap = null;
2752
2753    for (Option opt : options) {
2754      switch (opt) {
2755        case HBASE_VERSION:
2756          builder.setHBaseVersion(VersionInfo.getVersion());
2757          break;
2758        case CLUSTER_ID:
2759          builder.setClusterId(getClusterId());
2760          break;
2761        case MASTER:
2762          builder.setMasterName(getServerName());
2763          break;
2764        case BACKUP_MASTERS:
2765          builder.setBackerMasterNames(getBackupMasters());
2766          break;
2767        case TASKS: {
2768          // Master tasks
2769          builder.setMasterTasks(TaskMonitor.get().getTasks().stream()
2770            .map(task -> ServerTaskBuilder.newBuilder().setDescription(task.getDescription())
2771              .setStatus(task.getStatus())
2772              .setState(ServerTask.State.valueOf(task.getState().name()))
2773              .setStartTime(task.getStartTime()).setCompletionTime(task.getCompletionTimestamp())
2774              .build())
2775            .collect(Collectors.toList()));
2776          // TASKS is also synonymous with LIVE_SERVERS for now because task information for
2777          // regionservers is carried in ServerLoad.
2778          // Add entries to serverMetricsMap for all live servers, if we haven't already done so
2779          if (serverMetricsMap == null) {
2780            serverMetricsMap = getOnlineServers();
2781          }
2782          break;
2783        }
2784        case LIVE_SERVERS: {
2785          // Add entries to serverMetricsMap for all live servers, if we haven't already done so
2786          if (serverMetricsMap == null) {
2787            serverMetricsMap = getOnlineServers();
2788          }
2789          break;
2790        }
2791        case DEAD_SERVERS: {
2792          if (serverManager != null) {
2793            builder.setDeadServerNames(
2794              new ArrayList<>(serverManager.getDeadServers().copyServerNames()));
2795          }
2796          break;
2797        }
2798        case MASTER_COPROCESSORS: {
2799          if (cpHost != null) {
2800            builder.setMasterCoprocessorNames(Arrays.asList(getMasterCoprocessors()));
2801          }
2802          break;
2803        }
2804        case REGIONS_IN_TRANSITION: {
2805          if (assignmentManager != null) {
2806            builder.setRegionsInTransition(
2807              assignmentManager.getRegionStates().getRegionsStateInTransition());
2808          }
2809          break;
2810        }
2811        case BALANCER_ON: {
2812          if (loadBalancerTracker != null) {
2813            builder.setBalancerOn(loadBalancerTracker.isBalancerOn());
2814          }
2815          break;
2816        }
2817        case MASTER_INFO_PORT: {
2818          if (infoServer != null) {
2819            builder.setMasterInfoPort(infoServer.getPort());
2820          }
2821          break;
2822        }
2823        case SERVERS_NAME: {
2824          if (serverManager != null) {
2825            builder.setServerNames(serverManager.getOnlineServersList());
2826          }
2827          break;
2828        }
2829        case TABLE_TO_REGIONS_COUNT: {
2830          if (isActiveMaster() && isInitialized() && assignmentManager != null) {
2831            try {
2832              Map<TableName, RegionStatesCount> tableRegionStatesCountMap = new HashMap<>();
2833              Map<String, TableDescriptor> tableDescriptorMap = getTableDescriptors().getAll();
2834              for (TableDescriptor tableDescriptor : tableDescriptorMap.values()) {
2835                TableName tableName = tableDescriptor.getTableName();
2836                RegionStatesCount regionStatesCount =
2837                  assignmentManager.getRegionStatesCount(tableName);
2838                tableRegionStatesCountMap.put(tableName, regionStatesCount);
2839              }
2840              builder.setTableRegionStatesCount(tableRegionStatesCountMap);
2841            } catch (IOException e) {
2842              LOG.error("Error while populating TABLE_TO_REGIONS_COUNT for Cluster Metrics..", e);
2843            }
2844          }
2845          break;
2846        }
2847      }
2848    }
2849
2850    if (serverMetricsMap != null) {
2851      builder.setLiveServerMetrics(serverMetricsMap);
2852    }
2853
2854    return builder.build();
2855  }
2856
2857  private Map<ServerName, ServerMetrics> getOnlineServers() {
2858    if (serverManager != null) {
2859      final Map<ServerName, ServerMetrics> map = new HashMap<>();
2860      serverManager.getOnlineServers().entrySet().forEach(e -> map.put(e.getKey(), e.getValue()));
2861      return map;
2862    }
2863    return null;
2864  }
2865
2866  /**
2867   * @return cluster status
2868   */
2869  public ClusterMetrics getClusterMetrics() throws IOException {
2870    return getClusterMetrics(EnumSet.allOf(Option.class));
2871  }
2872
2873  public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
2874    if (cpHost != null) {
2875      cpHost.preGetClusterMetrics();
2876    }
2877    ClusterMetrics status = getClusterMetricsWithoutCoprocessor(options);
2878    if (cpHost != null) {
2879      cpHost.postGetClusterMetrics(status);
2880    }
2881    return status;
2882  }
2883
2884  /**
2885   * @return info port of active master or 0 if any exception occurs.
2886   */
2887  public int getActiveMasterInfoPort() {
2888    return activeMasterManager.getActiveMasterInfoPort();
2889  }
2890
2891  /**
2892   * @param sn is ServerName of the backup master
2893   * @return info port of backup master or 0 if any exception occurs.
2894   */
2895  public int getBackupMasterInfoPort(final ServerName sn) {
2896    return activeMasterManager.getBackupMasterInfoPort(sn);
2897  }
2898
2899  /**
2900   * The set of loaded coprocessors is stored in a static set. Since it's statically allocated, it
2901   * does not require that HMaster's cpHost be initialized prior to accessing it.
2902   * @return a String representation of the set of names of the loaded coprocessors.
2903   */
2904  public static String getLoadedCoprocessors() {
2905    return CoprocessorHost.getLoadedCoprocessors().toString();
2906  }
2907
2908  /**
2909   * @return timestamp in millis when HMaster was started.
2910   */
2911  public long getMasterStartTime() {
2912    return startcode;
2913  }
2914
2915  /**
2916   * @return timestamp in millis when HMaster became the active master.
2917   */
2918  public long getMasterActiveTime() {
2919    return masterActiveTime;
2920  }
2921
2922  /**
2923   * @return timestamp in millis when HMaster finished becoming the active master
2924   */
2925  public long getMasterFinishedInitializationTime() {
2926    return masterFinishedInitializationTime;
2927  }
2928
2929  public int getNumWALFiles() {
2930    return 0;
2931  }
2932
2933  public ProcedureStore getProcedureStore() {
2934    return procedureStore;
2935  }
2936
2937  public int getRegionServerInfoPort(final ServerName sn) {
2938    int port = this.serverManager.getInfoPort(sn);
2939    return port == 0
2940      ? conf.getInt(HConstants.REGIONSERVER_INFO_PORT, HConstants.DEFAULT_REGIONSERVER_INFOPORT)
2941      : port;
2942  }
2943
2944  @Override
2945  public String getRegionServerVersion(ServerName sn) {
2946    // Will return "0.0.0" if the server is not online to prevent move system region to unknown
2947    // version RS.
2948    return this.serverManager.getVersion(sn);
2949  }
2950
2951  @Override
2952  public void checkIfShouldMoveSystemRegionAsync() {
2953    assignmentManager.checkIfShouldMoveSystemRegionAsync();
2954  }
2955
2956  /**
2957   * @return array of coprocessor SimpleNames.
2958   */
2959  public String[] getMasterCoprocessors() {
2960    Set<String> masterCoprocessors = getMasterCoprocessorHost().getCoprocessors();
2961    return masterCoprocessors.toArray(new String[masterCoprocessors.size()]);
2962  }
2963
2964  @Override
2965  public void abort(String reason, Throwable cause) {
2966    if (!setAbortRequested() || isStopped()) {
2967      LOG.debug("Abort called but aborted={}, stopped={}", isAborted(), isStopped());
2968      return;
2969    }
2970    if (cpHost != null) {
2971      // HBASE-4014: dump a list of loaded coprocessors.
2972      LOG.error(HBaseMarkers.FATAL,
2973        "Master server abort: loaded coprocessors are: " + getLoadedCoprocessors());
2974    }
2975    String msg = "***** ABORTING master " + this + ": " + reason + " *****";
2976    if (cause != null) {
2977      LOG.error(HBaseMarkers.FATAL, msg, cause);
2978    } else {
2979      LOG.error(HBaseMarkers.FATAL, msg);
2980    }
2981
2982    try {
2983      stopMaster();
2984    } catch (IOException e) {
2985      LOG.error("Exception occurred while stopping master", e);
2986    }
2987  }
2988
2989  @Override
2990  public MasterCoprocessorHost getMasterCoprocessorHost() {
2991    return cpHost;
2992  }
2993
2994  @Override
2995  public MasterQuotaManager getMasterQuotaManager() {
2996    return quotaManager;
2997  }
2998
2999  @Override
3000  public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
3001    return procedureExecutor;
3002  }
3003
3004  @Override
3005  public ServerName getServerName() {
3006    return this.serverName;
3007  }
3008
3009  @Override
3010  public AssignmentManager getAssignmentManager() {
3011    return this.assignmentManager;
3012  }
3013
3014  @Override
3015  public CatalogJanitor getCatalogJanitor() {
3016    return this.catalogJanitorChore;
3017  }
3018
3019  public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
3020    return rsFatals;
3021  }
3022
3023  /**
3024   * Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
3025   */
3026  public void shutdown() throws IOException {
3027    if (cpHost != null) {
3028      cpHost.preShutdown();
3029    }
3030
3031    // Tell the servermanager cluster shutdown has been called. This makes it so when Master is
3032    // last running server, it'll stop itself. Next, we broadcast the cluster shutdown by setting
3033    // the cluster status as down. RegionServers will notice this change in state and will start
3034    // shutting themselves down. When last has exited, Master can go down.
3035    if (this.serverManager != null) {
3036      this.serverManager.shutdownCluster();
3037    }
3038    if (this.clusterStatusTracker != null) {
3039      try {
3040        this.clusterStatusTracker.setClusterDown();
3041      } catch (KeeperException e) {
3042        LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
3043      }
3044    }
3045    // Stop the procedure executor. Will stop any ongoing assign, unassign, server crash etc.,
3046    // processing so we can go down.
3047    if (this.procedureExecutor != null) {
3048      this.procedureExecutor.stop();
3049    }
3050    // Shutdown our cluster connection. This will kill any hosted RPCs that might be going on;
3051    // this is what we want especially if the Master is in startup phase doing call outs to
3052    // hbase:meta, etc. when cluster is down. Without ths connection close, we'd have to wait on
3053    // the rpc to timeout.
3054    if (this.asyncClusterConnection != null) {
3055      this.asyncClusterConnection.close();
3056    }
3057  }
3058
3059  public void stopMaster() throws IOException {
3060    if (cpHost != null) {
3061      cpHost.preStopMaster();
3062    }
3063    stop("Stopped by " + Thread.currentThread().getName());
3064  }
3065
3066  @Override
3067  public void stop(String msg) {
3068    if (!this.stopped) {
3069      LOG.info("***** STOPPING master '" + this + "' *****");
3070      this.stopped = true;
3071      LOG.info("STOPPED: " + msg);
3072      // Wakes run() if it is sleeping
3073      sleeper.skipSleepCycle();
3074      if (this.activeMasterManager != null) {
3075        this.activeMasterManager.stop();
3076      }
3077    }
3078  }
3079
3080  protected void checkServiceStarted() throws ServerNotRunningYetException {
3081    if (!serviceStarted) {
3082      throw new ServerNotRunningYetException("Server is not running yet");
3083    }
3084  }
3085
3086  void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException,
3087    MasterNotRunningException, MasterStoppedException {
3088    checkServiceStarted();
3089    if (!isInitialized()) {
3090      throw new PleaseHoldException("Master is initializing");
3091    }
3092    if (isStopped()) {
3093      throw new MasterStoppedException();
3094    }
3095  }
3096
3097  /**
3098   * Report whether this master is currently the active master or not. If not active master, we are
3099   * parked on ZK waiting to become active. This method is used for testing.
3100   * @return true if active master, false if not.
3101   */
3102  @Override
3103  public boolean isActiveMaster() {
3104    return activeMaster;
3105  }
3106
3107  /**
3108   * Report whether this master has completed with its initialization and is ready. If ready, the
3109   * master is also the active master. A standby master is never ready. This method is used for
3110   * testing.
3111   * @return true if master is ready to go, false if not.
3112   */
3113  @Override
3114  public boolean isInitialized() {
3115    return initialized.isReady();
3116  }
3117
3118  /**
3119   * Report whether this master is started This method is used for testing.
3120   * @return true if master is ready to go, false if not.
3121   */
3122  public boolean isOnline() {
3123    return serviceStarted;
3124  }
3125
3126  /**
3127   * Report whether this master is in maintenance mode.
3128   * @return true if master is in maintenanceMode
3129   */
3130  @Override
3131  public boolean isInMaintenanceMode() {
3132    return maintenanceMode;
3133  }
3134
3135  public void setInitialized(boolean isInitialized) {
3136    procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized);
3137  }
3138
3139  @Override
3140  public ProcedureEvent<?> getInitializedEvent() {
3141    return initialized;
3142  }
3143
3144  /**
3145   * Compute the average load across all region servers. Currently, this uses a very naive
3146   * computation - just uses the number of regions being served, ignoring stats about number of
3147   * requests.
3148   * @return the average load
3149   */
3150  public double getAverageLoad() {
3151    if (this.assignmentManager == null) {
3152      return 0;
3153    }
3154
3155    RegionStates regionStates = this.assignmentManager.getRegionStates();
3156    if (regionStates == null) {
3157      return 0;
3158    }
3159    return regionStates.getAverageLoad();
3160  }
3161
3162  @Override
3163  public boolean registerService(Service instance) {
3164    /*
3165     * No stacking of instances is allowed for a single service name
3166     */
3167    Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
3168    String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
3169    if (coprocessorServiceHandlers.containsKey(serviceName)) {
3170      LOG.error("Coprocessor service " + serviceName
3171        + " already registered, rejecting request from " + instance);
3172      return false;
3173    }
3174
3175    coprocessorServiceHandlers.put(serviceName, instance);
3176    if (LOG.isDebugEnabled()) {
3177      LOG.debug("Registered master coprocessor service: service=" + serviceName);
3178    }
3179    return true;
3180  }
3181
3182  /**
3183   * Utility for constructing an instance of the passed HMaster class. n * @return HMaster instance.
3184   */
3185  public static HMaster constructMaster(Class<? extends HMaster> masterClass,
3186    final Configuration conf) {
3187    try {
3188      Constructor<? extends HMaster> c = masterClass.getConstructor(Configuration.class);
3189      return c.newInstance(conf);
3190    } catch (Exception e) {
3191      Throwable error = e;
3192      if (
3193        e instanceof InvocationTargetException
3194          && ((InvocationTargetException) e).getTargetException() != null
3195      ) {
3196        error = ((InvocationTargetException) e).getTargetException();
3197      }
3198      throw new RuntimeException("Failed construction of Master: " + masterClass.toString() + ". ",
3199        error);
3200    }
3201  }
3202
3203  /**
3204   * @see org.apache.hadoop.hbase.master.HMasterCommandLine
3205   */
3206  public static void main(String[] args) {
3207    LOG.info("STARTING service " + HMaster.class.getSimpleName());
3208    VersionInfo.logVersion();
3209    new HMasterCommandLine(HMaster.class).doMain(args);
3210  }
3211
3212  public HFileCleaner getHFileCleaner() {
3213    return this.hfileCleaner;
3214  }
3215
3216  public LogCleaner getLogCleaner() {
3217    return this.logCleaner;
3218  }
3219
3220  /**
3221   * @return the underlying snapshot manager
3222   */
3223  @Override
3224  public SnapshotManager getSnapshotManager() {
3225    return this.snapshotManager;
3226  }
3227
3228  /**
3229   * @return the underlying MasterProcedureManagerHost
3230   */
3231  @Override
3232  public MasterProcedureManagerHost getMasterProcedureManagerHost() {
3233    return mpmHost;
3234  }
3235
3236  @Override
3237  public ClusterSchema getClusterSchema() {
3238    return this.clusterSchemaService;
3239  }
3240
3241  /**
3242   * Create a new Namespace.
3243   * @param namespaceDescriptor descriptor for new Namespace
3244   * @param nonceGroup          Identifier for the source of the request, a client or process.
3245   * @param nonce               A unique identifier for this operation from the client or process
3246   *                            identified by <code>nonceGroup</code> (the source must ensure each
3247   *                            operation gets a unique id).
3248   * @return procedure id
3249   */
3250  long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
3251    final long nonce) throws IOException {
3252    checkInitialized();
3253
3254    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
3255
3256    return MasterProcedureUtil
3257      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
3258        @Override
3259        protected void run() throws IOException {
3260          getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor);
3261          // We need to wait for the procedure to potentially fail due to "prepare" sanity
3262          // checks. This will block only the beginning of the procedure. See HBASE-19953.
3263          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3264          LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
3265          // Execute the operation synchronously - wait for the operation to complete before
3266          // continuing.
3267          setProcId(getClusterSchema().createNamespace(namespaceDescriptor, getNonceKey(), latch));
3268          latch.await();
3269          getMaster().getMasterCoprocessorHost().postCreateNamespace(namespaceDescriptor);
3270        }
3271
3272        @Override
3273        protected String getDescription() {
3274          return "CreateNamespaceProcedure";
3275        }
3276      });
3277  }
3278
3279  /**
3280   * Modify an existing Namespace.
3281   * @param nonceGroup Identifier for the source of the request, a client or process.
3282   * @param nonce      A unique identifier for this operation from the client or process identified
3283   *                   by <code>nonceGroup</code> (the source must ensure each operation gets a
3284   *                   unique id).
3285   * @return procedure id
3286   */
3287  long modifyNamespace(final NamespaceDescriptor newNsDescriptor, final long nonceGroup,
3288    final long nonce) throws IOException {
3289    checkInitialized();
3290
3291    TableName.isLegalNamespaceName(Bytes.toBytes(newNsDescriptor.getName()));
3292
3293    return MasterProcedureUtil
3294      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
3295        @Override
3296        protected void run() throws IOException {
3297          NamespaceDescriptor oldNsDescriptor = getNamespace(newNsDescriptor.getName());
3298          getMaster().getMasterCoprocessorHost().preModifyNamespace(oldNsDescriptor,
3299            newNsDescriptor);
3300          // We need to wait for the procedure to potentially fail due to "prepare" sanity
3301          // checks. This will block only the beginning of the procedure. See HBASE-19953.
3302          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3303          LOG.info(getClientIdAuditPrefix() + " modify " + newNsDescriptor);
3304          // Execute the operation synchronously - wait for the operation to complete before
3305          // continuing.
3306          setProcId(getClusterSchema().modifyNamespace(newNsDescriptor, getNonceKey(), latch));
3307          latch.await();
3308          getMaster().getMasterCoprocessorHost().postModifyNamespace(oldNsDescriptor,
3309            newNsDescriptor);
3310        }
3311
3312        @Override
3313        protected String getDescription() {
3314          return "ModifyNamespaceProcedure";
3315        }
3316      });
3317  }
3318
3319  /**
3320   * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
3321   * @param nonceGroup Identifier for the source of the request, a client or process.
3322   * @param nonce      A unique identifier for this operation from the client or process identified
3323   *                   by <code>nonceGroup</code> (the source must ensure each operation gets a
3324   *                   unique id).
3325   * @return procedure id
3326   */
3327  long deleteNamespace(final String name, final long nonceGroup, final long nonce)
3328    throws IOException {
3329    checkInitialized();
3330
3331    return MasterProcedureUtil
3332      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
3333        @Override
3334        protected void run() throws IOException {
3335          getMaster().getMasterCoprocessorHost().preDeleteNamespace(name);
3336          LOG.info(getClientIdAuditPrefix() + " delete " + name);
3337          // Execute the operation synchronously - wait for the operation to complete before
3338          // continuing.
3339          //
3340          // We need to wait for the procedure to potentially fail due to "prepare" sanity
3341          // checks. This will block only the beginning of the procedure. See HBASE-19953.
3342          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3343          setProcId(submitProcedure(
3344            new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name, latch)));
3345          latch.await();
3346          // Will not be invoked in the face of Exception thrown by the Procedure's execution
3347          getMaster().getMasterCoprocessorHost().postDeleteNamespace(name);
3348        }
3349
3350        @Override
3351        protected String getDescription() {
3352          return "DeleteNamespaceProcedure";
3353        }
3354      });
3355  }
3356
3357  /**
3358   * Get a Namespace
3359   * @param name Name of the Namespace
3360   * @return Namespace descriptor for <code>name</code>
3361   */
3362  NamespaceDescriptor getNamespace(String name) throws IOException {
3363    checkInitialized();
3364    if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);
3365    NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);
3366    if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);
3367    return nsd;
3368  }
3369
3370  /**
3371   * Get all Namespaces
3372   * @return All Namespace descriptors
3373   */
3374  List<NamespaceDescriptor> getNamespaces() throws IOException {
3375    checkInitialized();
3376    final List<NamespaceDescriptor> nsds = new ArrayList<>();
3377    if (cpHost != null) {
3378      cpHost.preListNamespaceDescriptors(nsds);
3379    }
3380    nsds.addAll(this.clusterSchemaService.getNamespaces());
3381    if (this.cpHost != null) {
3382      this.cpHost.postListNamespaceDescriptors(nsds);
3383    }
3384    return nsds;
3385  }
3386
3387  /**
3388   * List namespace names
3389   * @return All namespace names
3390   */
3391  public List<String> listNamespaces() throws IOException {
3392    checkInitialized();
3393    List<String> namespaces = new ArrayList<>();
3394    if (cpHost != null) {
3395      cpHost.preListNamespaces(namespaces);
3396    }
3397    for (NamespaceDescriptor namespace : clusterSchemaService.getNamespaces()) {
3398      namespaces.add(namespace.getName());
3399    }
3400    if (cpHost != null) {
3401      cpHost.postListNamespaces(namespaces);
3402    }
3403    return namespaces;
3404  }
3405
3406  @Override
3407  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
3408    checkInitialized();
3409    return listTableNames(name, null, true);
3410  }
3411
3412  @Override
3413  public List<TableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
3414    checkInitialized();
3415    return listTableDescriptors(name, null, null, true);
3416  }
3417
3418  @Override
3419  public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
3420    throws IOException {
3421    if (cpHost != null) {
3422      cpHost.preAbortProcedure(this.procedureExecutor, procId);
3423    }
3424
3425    final boolean result = this.procedureExecutor.abort(procId, mayInterruptIfRunning);
3426
3427    if (cpHost != null) {
3428      cpHost.postAbortProcedure();
3429    }
3430
3431    return result;
3432  }
3433
3434  @Override
3435  public List<Procedure<?>> getProcedures() throws IOException {
3436    if (cpHost != null) {
3437      cpHost.preGetProcedures();
3438    }
3439
3440    @SuppressWarnings({ "unchecked", "rawtypes" })
3441    List<Procedure<?>> procList = (List) this.procedureExecutor.getProcedures();
3442
3443    if (cpHost != null) {
3444      cpHost.postGetProcedures(procList);
3445    }
3446
3447    return procList;
3448  }
3449
3450  @Override
3451  public List<LockedResource> getLocks() throws IOException {
3452    if (cpHost != null) {
3453      cpHost.preGetLocks();
3454    }
3455
3456    MasterProcedureScheduler procedureScheduler =
3457      procedureExecutor.getEnvironment().getProcedureScheduler();
3458
3459    final List<LockedResource> lockedResources = procedureScheduler.getLocks();
3460
3461    if (cpHost != null) {
3462      cpHost.postGetLocks(lockedResources);
3463    }
3464
3465    return lockedResources;
3466  }
3467
3468  /**
3469   * Returns the list of table descriptors that match the specified request
3470   * @param namespace        the namespace to query, or null if querying for all
3471   * @param regex            The regular expression to match against, or null if querying for all
3472   * @param tableNameList    the list of table names, or null if querying for all
3473   * @param includeSysTables False to match only against userspace tables
3474   * @return the list of table descriptors
3475   */
3476  public List<TableDescriptor> listTableDescriptors(final String namespace, final String regex,
3477    final List<TableName> tableNameList, final boolean includeSysTables) throws IOException {
3478    List<TableDescriptor> htds = new ArrayList<>();
3479    if (cpHost != null) {
3480      cpHost.preGetTableDescriptors(tableNameList, htds, regex);
3481    }
3482    htds = getTableDescriptors(htds, namespace, regex, tableNameList, includeSysTables);
3483    if (cpHost != null) {
3484      cpHost.postGetTableDescriptors(tableNameList, htds, regex);
3485    }
3486    return htds;
3487  }
3488
3489  /**
3490   * Returns the list of table names that match the specified request
3491   * @param regex            The regular expression to match against, or null if querying for all
3492   * @param namespace        the namespace to query, or null if querying for all
3493   * @param includeSysTables False to match only against userspace tables
3494   * @return the list of table names
3495   */
3496  public List<TableName> listTableNames(final String namespace, final String regex,
3497    final boolean includeSysTables) throws IOException {
3498    List<TableDescriptor> htds = new ArrayList<>();
3499    if (cpHost != null) {
3500      cpHost.preGetTableNames(htds, regex);
3501    }
3502    htds = getTableDescriptors(htds, namespace, regex, null, includeSysTables);
3503    if (cpHost != null) {
3504      cpHost.postGetTableNames(htds, regex);
3505    }
3506    List<TableName> result = new ArrayList<>(htds.size());
3507    for (TableDescriptor htd : htds)
3508      result.add(htd.getTableName());
3509    return result;
3510  }
3511
3512  /**
3513   * Return a list of table table descriptors after applying any provided filter parameters. Note
3514   * that the user-facing description of this filter logic is presented on the class-level javadoc
3515   * of {@link NormalizeTableFilterParams}.
3516   */
3517  private List<TableDescriptor> getTableDescriptors(final List<TableDescriptor> htds,
3518    final String namespace, final String regex, final List<TableName> tableNameList,
3519    final boolean includeSysTables) throws IOException {
3520    if (tableNameList == null || tableNameList.isEmpty()) {
3521      // request for all TableDescriptors
3522      Collection<TableDescriptor> allHtds;
3523      if (namespace != null && namespace.length() > 0) {
3524        // Do a check on the namespace existence. Will fail if does not exist.
3525        this.clusterSchemaService.getNamespace(namespace);
3526        allHtds = tableDescriptors.getByNamespace(namespace).values();
3527      } else {
3528        allHtds = tableDescriptors.getAll().values();
3529      }
3530      for (TableDescriptor desc : allHtds) {
3531        if (
3532          tableStateManager.isTablePresent(desc.getTableName())
3533            && (includeSysTables || !desc.getTableName().isSystemTable())
3534        ) {
3535          htds.add(desc);
3536        }
3537      }
3538    } else {
3539      for (TableName s : tableNameList) {
3540        if (tableStateManager.isTablePresent(s)) {
3541          TableDescriptor desc = tableDescriptors.get(s);
3542          if (desc != null) {
3543            htds.add(desc);
3544          }
3545        }
3546      }
3547    }
3548
3549    // Retains only those matched by regular expression.
3550    if (regex != null) filterTablesByRegex(htds, Pattern.compile(regex));
3551    return htds;
3552  }
3553
3554  /**
3555   * Removes the table descriptors that don't match the pattern.
3556   * @param descriptors list of table descriptors to filter
3557   * @param pattern     the regex to use
3558   */
3559  private static void filterTablesByRegex(final Collection<TableDescriptor> descriptors,
3560    final Pattern pattern) {
3561    final String defaultNS = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
3562    Iterator<TableDescriptor> itr = descriptors.iterator();
3563    while (itr.hasNext()) {
3564      TableDescriptor htd = itr.next();
3565      String tableName = htd.getTableName().getNameAsString();
3566      boolean matched = pattern.matcher(tableName).matches();
3567      if (!matched && htd.getTableName().getNamespaceAsString().equals(defaultNS)) {
3568        matched = pattern.matcher(defaultNS + TableName.NAMESPACE_DELIM + tableName).matches();
3569      }
3570      if (!matched) {
3571        itr.remove();
3572      }
3573    }
3574  }
3575
3576  @Override
3577  public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
3578    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
3579      .getLastMajorCompactionTimestamp(table);
3580  }
3581
3582  @Override
3583  public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
3584    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
3585      .getLastMajorCompactionTimestamp(regionName);
3586  }
3587
3588  /**
3589   * Gets the mob file compaction state for a specific table. Whether all the mob files are selected
3590   * is known during the compaction execution, but the statistic is done just before compaction
3591   * starts, it is hard to know the compaction type at that time, so the rough statistics are chosen
3592   * for the mob file compaction. Only two compaction states are available,
3593   * CompactionState.MAJOR_AND_MINOR and CompactionState.NONE.
3594   * @param tableName The current table name.
3595   * @return If a given table is in mob file compaction now.
3596   */
3597  public GetRegionInfoResponse.CompactionState getMobCompactionState(TableName tableName) {
3598    AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3599    if (compactionsCount != null && compactionsCount.get() != 0) {
3600      return GetRegionInfoResponse.CompactionState.MAJOR_AND_MINOR;
3601    }
3602    return GetRegionInfoResponse.CompactionState.NONE;
3603  }
3604
3605  public void reportMobCompactionStart(TableName tableName) throws IOException {
3606    IdLock.Entry lockEntry = null;
3607    try {
3608      lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
3609      AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3610      if (compactionsCount == null) {
3611        compactionsCount = new AtomicInteger(0);
3612        mobCompactionStates.put(tableName, compactionsCount);
3613      }
3614      compactionsCount.incrementAndGet();
3615    } finally {
3616      if (lockEntry != null) {
3617        mobCompactionLock.releaseLockEntry(lockEntry);
3618      }
3619    }
3620  }
3621
3622  public void reportMobCompactionEnd(TableName tableName) throws IOException {
3623    IdLock.Entry lockEntry = null;
3624    try {
3625      lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
3626      AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3627      if (compactionsCount != null) {
3628        int count = compactionsCount.decrementAndGet();
3629        // remove the entry if the count is 0.
3630        if (count == 0) {
3631          mobCompactionStates.remove(tableName);
3632        }
3633      }
3634    } finally {
3635      if (lockEntry != null) {
3636        mobCompactionLock.releaseLockEntry(lockEntry);
3637      }
3638    }
3639  }
3640
3641  /**
3642   * Queries the state of the {@link LoadBalancerTracker}. If the balancer is not initialized, false
3643   * is returned.
3644   * @return The state of the load balancer, or false if the load balancer isn't defined.
3645   */
3646  public boolean isBalancerOn() {
3647    return !isInMaintenanceMode() && loadBalancerTracker != null
3648      && loadBalancerTracker.isBalancerOn();
3649  }
3650
3651  /**
3652   * Queries the state of the {@link RegionNormalizerTracker}. If it's not initialized, false is
3653   * returned.
3654   */
3655  public boolean isNormalizerOn() {
3656    return !isInMaintenanceMode() && getRegionNormalizerManager().isNormalizerOn();
3657  }
3658
3659  /**
3660   * Queries the state of the {@link SplitOrMergeTracker}. If it is not initialized, false is
3661   * returned. If switchType is illegal, false will return.
3662   * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}
3663   * @return The state of the switch
3664   */
3665  @Override
3666  public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
3667    return !isInMaintenanceMode() && splitOrMergeTracker != null
3668      && splitOrMergeTracker.isSplitOrMergeEnabled(switchType);
3669  }
3670
3671  /**
3672   * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.
3673   * <p/>
3674   * Notice that, the base load balancer will always be {@link RSGroupBasedLoadBalancer} now, so
3675   * this method will return the balancer used inside each rs group.
3676   * @return The name of the {@link LoadBalancer} in use.
3677   */
3678  public String getLoadBalancerClassName() {
3679    return conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
3680      LoadBalancerFactory.getDefaultLoadBalancerClass().getName());
3681  }
3682
3683  public SplitOrMergeTracker getSplitOrMergeTracker() {
3684    return splitOrMergeTracker;
3685  }
3686
3687  @Override
3688  public RSGroupBasedLoadBalancer getLoadBalancer() {
3689    return balancer;
3690  }
3691
3692  @Override
3693  public FavoredNodesManager getFavoredNodesManager() {
3694    return balancer.getFavoredNodesManager();
3695  }
3696
3697  private long executePeerProcedure(AbstractPeerProcedure<?> procedure) throws IOException {
3698    long procId = procedureExecutor.submitProcedure(procedure);
3699    procedure.getLatch().await();
3700    return procId;
3701  }
3702
3703  @Override
3704  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
3705    throws ReplicationException, IOException {
3706    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config="
3707      + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
3708    return executePeerProcedure(new AddPeerProcedure(peerId, peerConfig, enabled));
3709  }
3710
3711  @Override
3712  public long removeReplicationPeer(String peerId) throws ReplicationException, IOException {
3713    LOG.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId);
3714    return executePeerProcedure(new RemovePeerProcedure(peerId));
3715  }
3716
3717  @Override
3718  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
3719    LOG.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId);
3720    return executePeerProcedure(new EnablePeerProcedure(peerId));
3721  }
3722
3723  @Override
3724  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
3725    LOG.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId);
3726    return executePeerProcedure(new DisablePeerProcedure(peerId));
3727  }
3728
3729  @Override
3730  public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
3731    throws ReplicationException, IOException {
3732    if (cpHost != null) {
3733      cpHost.preGetReplicationPeerConfig(peerId);
3734    }
3735    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId);
3736    ReplicationPeerConfig peerConfig = this.replicationPeerManager.getPeerConfig(peerId)
3737      .orElseThrow(() -> new ReplicationPeerNotFoundException(peerId));
3738    if (cpHost != null) {
3739      cpHost.postGetReplicationPeerConfig(peerId);
3740    }
3741    return peerConfig;
3742  }
3743
3744  @Override
3745  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
3746    throws ReplicationException, IOException {
3747    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId
3748      + ", config=" + peerConfig);
3749    return executePeerProcedure(new UpdatePeerConfigProcedure(peerId, peerConfig));
3750  }
3751
3752  @Override
3753  public List<ReplicationPeerDescription> listReplicationPeers(String regex)
3754    throws ReplicationException, IOException {
3755    if (cpHost != null) {
3756      cpHost.preListReplicationPeers(regex);
3757    }
3758    LOG.debug("{} list replication peers, regex={}", getClientIdAuditPrefix(), regex);
3759    Pattern pattern = regex == null ? null : Pattern.compile(regex);
3760    List<ReplicationPeerDescription> peers = this.replicationPeerManager.listPeers(pattern);
3761    if (cpHost != null) {
3762      cpHost.postListReplicationPeers(regex);
3763    }
3764    return peers;
3765  }
3766
3767  @Override
3768  public long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
3769    throws ReplicationException, IOException {
3770    LOG.info(
3771      getClientIdAuditPrefix()
3772        + " transit current cluster state to {} in a synchronous replication peer id={}",
3773      state, peerId);
3774    return executePeerProcedure(new TransitPeerSyncReplicationStateProcedure(peerId, state));
3775  }
3776
3777  /**
3778   * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
3779   * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0
3780   * @param servers Region servers to decommission.
3781   */
3782  public void decommissionRegionServers(final List<ServerName> servers, final boolean offload)
3783    throws IOException {
3784    List<ServerName> serversAdded = new ArrayList<>(servers.size());
3785    // Place the decommission marker first.
3786    String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;
3787    for (ServerName server : servers) {
3788      try {
3789        String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
3790        ZKUtil.createAndFailSilent(getZooKeeper(), node);
3791      } catch (KeeperException ke) {
3792        throw new HBaseIOException(
3793          this.zooKeeper.prefix("Unable to decommission '" + server.getServerName() + "'."), ke);
3794      }
3795      if (this.serverManager.addServerToDrainList(server)) {
3796        serversAdded.add(server);
3797      }
3798    }
3799    // Move the regions off the decommissioned servers.
3800    if (offload) {
3801      final List<ServerName> destServers = this.serverManager.createDestinationServersList();
3802      for (ServerName server : serversAdded) {
3803        final List<RegionInfo> regionsOnServer = this.assignmentManager.getRegionsOnServer(server);
3804        for (RegionInfo hri : regionsOnServer) {
3805          ServerName dest = balancer.randomAssignment(hri, destServers);
3806          if (dest == null) {
3807            throw new HBaseIOException("Unable to determine a plan to move " + hri);
3808          }
3809          RegionPlan rp = new RegionPlan(hri, server, dest);
3810          this.assignmentManager.moveAsync(rp);
3811        }
3812      }
3813    }
3814  }
3815
3816  /**
3817   * List region servers marked as decommissioned (previously called 'draining') to not get regions
3818   * assigned to them.
3819   * @return List of decommissioned servers.
3820   */
3821  public List<ServerName> listDecommissionedRegionServers() {
3822    return this.serverManager.getDrainingServersList();
3823  }
3824
3825  /**
3826   * Remove decommission marker (previously called 'draining') from a region server to allow regions
3827   * assignments. Load regions onto the server asynchronously if a list of regions is given
3828   * @param server Region server to remove decommission marker from.
3829   */
3830  public void recommissionRegionServer(final ServerName server,
3831    final List<byte[]> encodedRegionNames) throws IOException {
3832    // Remove the server from decommissioned (draining) server list.
3833    String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;
3834    String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
3835    try {
3836      ZKUtil.deleteNodeFailSilent(getZooKeeper(), node);
3837    } catch (KeeperException ke) {
3838      throw new HBaseIOException(
3839        this.zooKeeper.prefix("Unable to recommission '" + server.getServerName() + "'."), ke);
3840    }
3841    this.serverManager.removeServerFromDrainList(server);
3842
3843    // Load the regions onto the server if we are given a list of regions.
3844    if (encodedRegionNames == null || encodedRegionNames.isEmpty()) {
3845      return;
3846    }
3847    if (!this.serverManager.isServerOnline(server)) {
3848      return;
3849    }
3850    for (byte[] encodedRegionName : encodedRegionNames) {
3851      RegionState regionState =
3852        assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));
3853      if (regionState == null) {
3854        LOG.warn("Unknown region " + Bytes.toStringBinary(encodedRegionName));
3855        continue;
3856      }
3857      RegionInfo hri = regionState.getRegion();
3858      if (server.equals(regionState.getServerName())) {
3859        LOG.info("Skipping move of region " + hri.getRegionNameAsString()
3860          + " because region already assigned to the same server " + server + ".");
3861        continue;
3862      }
3863      RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), server);
3864      this.assignmentManager.moveAsync(rp);
3865    }
3866  }
3867
3868  @Override
3869  public LockManager getLockManager() {
3870    return lockManager;
3871  }
3872
3873  public QuotaObserverChore getQuotaObserverChore() {
3874    return this.quotaObserverChore;
3875  }
3876
3877  public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
3878    return this.spaceQuotaSnapshotNotifier;
3879  }
3880
3881  @SuppressWarnings("unchecked")
3882  private RemoteProcedure<MasterProcedureEnv, ?> getRemoteProcedure(long procId) {
3883    Procedure<?> procedure = procedureExecutor.getProcedure(procId);
3884    if (procedure == null) {
3885      return null;
3886    }
3887    assert procedure instanceof RemoteProcedure;
3888    return (RemoteProcedure<MasterProcedureEnv, ?>) procedure;
3889  }
3890
3891  public void remoteProcedureCompleted(long procId) {
3892    LOG.debug("Remote procedure done, pid={}", procId);
3893    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
3894    if (procedure != null) {
3895      procedure.remoteOperationCompleted(procedureExecutor.getEnvironment());
3896    }
3897  }
3898
3899  public void remoteProcedureFailed(long procId, RemoteProcedureException error) {
3900    LOG.debug("Remote procedure failed, pid={}", procId, error);
3901    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
3902    if (procedure != null) {
3903      procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
3904    }
3905  }
3906
3907  /**
3908   * Reopen regions provided in the argument
3909   * @param tableName   The current table name
3910   * @param regionNames The region names of the regions to reopen
3911   * @param nonceGroup  Identifier for the source of the request, a client or process
3912   * @param nonce       A unique identifier for this operation from the client or process identified
3913   *                    by <code>nonceGroup</code> (the source must ensure each operation gets a
3914   *                    unique id).
3915   * @return procedure Id
3916   * @throws IOException if reopening region fails while running procedure
3917   */
3918  long reopenRegions(final TableName tableName, final List<byte[]> regionNames,
3919    final long nonceGroup, final long nonce) throws IOException {
3920
3921    return MasterProcedureUtil
3922      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
3923
3924        @Override
3925        protected void run() throws IOException {
3926          submitProcedure(new ReopenTableRegionsProcedure(tableName, regionNames));
3927        }
3928
3929        @Override
3930        protected String getDescription() {
3931          return "ReopenTableRegionsProcedure";
3932        }
3933
3934      });
3935
3936  }
3937
3938  @Override
3939  public ReplicationPeerManager getReplicationPeerManager() {
3940    return replicationPeerManager;
3941  }
3942
3943  public HashMap<String, List<Pair<ServerName, ReplicationLoadSource>>>
3944    getReplicationLoad(ServerName[] serverNames) {
3945    List<ReplicationPeerDescription> peerList = this.getReplicationPeerManager().listPeers(null);
3946    if (peerList == null) {
3947      return null;
3948    }
3949    HashMap<String, List<Pair<ServerName, ReplicationLoadSource>>> replicationLoadSourceMap =
3950      new HashMap<>(peerList.size());
3951    peerList.stream()
3952      .forEach(peer -> replicationLoadSourceMap.put(peer.getPeerId(), new ArrayList<>()));
3953    for (ServerName serverName : serverNames) {
3954      List<ReplicationLoadSource> replicationLoadSources =
3955        getServerManager().getLoad(serverName).getReplicationLoadSourceList();
3956      for (ReplicationLoadSource replicationLoadSource : replicationLoadSources) {
3957        List<Pair<ServerName, ReplicationLoadSource>> replicationLoadSourceList =
3958          replicationLoadSourceMap.get(replicationLoadSource.getPeerID());
3959        if (replicationLoadSourceList == null) {
3960          LOG.debug("{} does not exist, but it exists "
3961            + "in znode(/hbase/replication/rs). when the rs restarts, peerId is deleted, so "
3962            + "we just need to ignore it", replicationLoadSource.getPeerID());
3963          continue;
3964        }
3965        replicationLoadSourceList.add(new Pair<>(serverName, replicationLoadSource));
3966      }
3967    }
3968    for (List<Pair<ServerName, ReplicationLoadSource>> loads : replicationLoadSourceMap.values()) {
3969      if (loads.size() > 0) {
3970        loads.sort(Comparator.comparingLong(load -> (-1) * load.getSecond().getReplicationLag()));
3971      }
3972    }
3973    return replicationLoadSourceMap;
3974  }
3975
3976  /**
3977   * This method modifies the master's configuration in order to inject replication-related features
3978   */
3979  @InterfaceAudience.Private
3980  public static void decorateMasterConfiguration(Configuration conf) {
3981    String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
3982    String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
3983    if (plugins == null || !plugins.contains(cleanerClass)) {
3984      conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
3985    }
3986    if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
3987      plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
3988      cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();
3989      if (!plugins.contains(cleanerClass)) {
3990        conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);
3991      }
3992    }
3993  }
3994
3995  public SnapshotQuotaObserverChore getSnapshotQuotaObserverChore() {
3996    return this.snapshotQuotaChore;
3997  }
3998
3999  public ActiveMasterManager getActiveMasterManager() {
4000    return activeMasterManager;
4001  }
4002
4003  @Override
4004  public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() {
4005    return this.syncReplicationReplayWALManager;
4006  }
4007
4008  public HbckChore getHbckChore() {
4009    return this.hbckChore;
4010  }
4011
4012  @Override
4013  public void runReplicationBarrierCleaner() {
4014    ReplicationBarrierCleaner rbc = this.replicationBarrierCleaner;
4015    if (rbc != null) {
4016      rbc.chore();
4017    }
4018  }
4019
4020  @Override
4021  public RSGroupInfoManager getRSGroupInfoManager() {
4022    return rsGroupInfoManager;
4023  }
4024
4025  /**
4026   * Get the compaction state of the table
4027   * @param tableName The table name
4028   * @return CompactionState Compaction state of the table
4029   */
4030  public CompactionState getCompactionState(final TableName tableName) {
4031    CompactionState compactionState = CompactionState.NONE;
4032    try {
4033      List<RegionInfo> regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);
4034      for (RegionInfo regionInfo : regions) {
4035        ServerName serverName =
4036          assignmentManager.getRegionStates().getRegionServerOfRegion(regionInfo);
4037        if (serverName == null) {
4038          continue;
4039        }
4040        ServerMetrics sl = serverManager.getLoad(serverName);
4041        if (sl == null) {
4042          continue;
4043        }
4044        RegionMetrics regionMetrics = sl.getRegionMetrics().get(regionInfo.getRegionName());
4045        if (regionMetrics.getCompactionState() == CompactionState.MAJOR) {
4046          if (compactionState == CompactionState.MINOR) {
4047            compactionState = CompactionState.MAJOR_AND_MINOR;
4048          } else {
4049            compactionState = CompactionState.MAJOR;
4050          }
4051        } else if (regionMetrics.getCompactionState() == CompactionState.MINOR) {
4052          if (compactionState == CompactionState.MAJOR) {
4053            compactionState = CompactionState.MAJOR_AND_MINOR;
4054          } else {
4055            compactionState = CompactionState.MINOR;
4056          }
4057        }
4058      }
4059    } catch (Exception e) {
4060      compactionState = null;
4061      LOG.error("Exception when get compaction state for " + tableName.getNameAsString(), e);
4062    }
4063    return compactionState;
4064  }
4065
4066  @Override
4067  public MetaLocationSyncer getMetaLocationSyncer() {
4068    return metaLocationSyncer;
4069  }
4070
4071  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4072      allowedOnPath = ".*/src/test/.*")
4073  public MasterRegion getMasterRegion() {
4074    return masterRegion;
4075  }
4076
4077  @Override
4078  public void onConfigurationChange(Configuration newConf) {
4079    try {
4080      Superusers.initialize(newConf);
4081    } catch (IOException e) {
4082      LOG.warn("Failed to initialize SuperUsers on reloading of the configuration");
4083    }
4084    // append the quotas observer back to the master coprocessor key
4085    setQuotasObserver(newConf);
4086    // update region server coprocessor if the configuration has changed.
4087    if (
4088      CoprocessorConfigurationUtil.checkConfigurationChange(getConfiguration(), newConf,
4089        CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY) && !maintenanceMode
4090    ) {
4091      LOG.info("Update the master coprocessor(s) because the configuration has changed");
4092      initializeCoprocessorHost(newConf);
4093    }
4094  }
4095
4096  @Override
4097  protected NamedQueueRecorder createNamedQueueRecord() {
4098    final boolean isBalancerDecisionRecording =
4099      conf.getBoolean(BaseLoadBalancer.BALANCER_DECISION_BUFFER_ENABLED,
4100        BaseLoadBalancer.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED);
4101    final boolean isBalancerRejectionRecording =
4102      conf.getBoolean(BaseLoadBalancer.BALANCER_REJECTION_BUFFER_ENABLED,
4103        BaseLoadBalancer.DEFAULT_BALANCER_REJECTION_BUFFER_ENABLED);
4104    if (isBalancerDecisionRecording || isBalancerRejectionRecording) {
4105      return NamedQueueRecorder.getInstance(conf);
4106    } else {
4107      return null;
4108    }
4109  }
4110
4111  @Override
4112  protected boolean clusterMode() {
4113    return true;
4114  }
4115
4116  public String getClusterId() {
4117    if (activeMaster) {
4118      return clusterId;
4119    }
4120    return cachedClusterId.getFromCacheOrFetch();
4121  }
4122
4123  public Optional<ServerName> getActiveMaster() {
4124    return activeMasterManager.getActiveMasterServerName();
4125  }
4126
4127  public List<ServerName> getBackupMasters() {
4128    return activeMasterManager.getBackupMasters();
4129  }
4130
4131  @Override
4132  public Iterator<ServerName> getBootstrapNodes() {
4133    return regionServerTracker.getRegionServers().iterator();
4134  }
4135
4136  @Override
4137  public List<HRegionLocation> getMetaLocations() {
4138    return metaRegionLocationCache.getMetaRegionLocations();
4139  }
4140
4141  public Collection<ServerName> getLiveRegionServers() {
4142    return regionServerTracker.getRegionServers();
4143  }
4144
4145  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4146      allowedOnPath = ".*/src/test/.*")
4147  void setLoadBalancer(RSGroupBasedLoadBalancer loadBalancer) {
4148    this.balancer = loadBalancer;
4149  }
4150
4151  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4152      allowedOnPath = ".*/src/test/.*")
4153  void setAssignmentManager(AssignmentManager assignmentManager) {
4154    this.assignmentManager = assignmentManager;
4155  }
4156
4157  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4158      allowedOnPath = ".*/src/test/.*")
4159  static void setDisableBalancerChoreForTest(boolean disable) {
4160    disableBalancerChoreForTest = disable;
4161  }
4162
4163  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4164      allowedOnPath = ".*/src/test/.*")
4165  public ConfigurationManager getConfigurationManager() {
4166    return configurationManager;
4167  }
4168
4169  private void setQuotasObserver(Configuration conf) {
4170    // Add the Observer to delete quotas on table deletion before starting all CPs by
4171    // default with quota support, avoiding if user specifically asks to not load this Observer.
4172    if (QuotaUtil.isQuotaEnabled(conf)) {
4173      updateConfigurationForQuotasObserver(conf);
4174    }
4175  }
4176
4177  private void initializeCoprocessorHost(Configuration conf) {
4178    // initialize master side coprocessors before we start handling requests
4179    this.cpHost = new MasterCoprocessorHost(this, conf);
4180  }
4181
4182}