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