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