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