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