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      if (initMetaProc.isFailed() && initMetaProc.hasException()) {
1126        throw new IOException("Failed to initialize meta table", initMetaProc.getException());
1127      }
1128    }
1129    // Wake up this server to check in
1130    sleeper.skipSleepCycle();
1131
1132    // Wait for region servers to report in.
1133    // With this as part of master initialization, it precludes our being able to start a single
1134    // server that is both Master and RegionServer. Needs more thought. TODO.
1135    String statusStr = "Wait for region servers to report in";
1136    MonitoredTask waitRegionServer = startupTaskGroup.addTask(statusStr);
1137    LOG.info(Objects.toString(waitRegionServer));
1138    waitForRegionServers(waitRegionServer);
1139
1140    // Check if master is shutting down because issue initializing regionservers or balancer.
1141    if (isStopped()) {
1142      return;
1143    }
1144
1145    startupTaskGroup.addTask("Starting assignment manager");
1146    // FIRST HBASE:META READ!!!!
1147    // The below cannot make progress w/o hbase:meta being online.
1148    // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
1149    // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
1150    // if it is down. It may take a while to come online. So, wait here until meta if for sure
1151    // available. That's what waitForMetaOnline does.
1152    if (!waitForMetaOnline()) {
1153      return;
1154    }
1155
1156    TableDescriptor metaDescriptor = tableDescriptors.get(TableName.META_TABLE_NAME);
1157    final ColumnFamilyDescriptor tableFamilyDesc =
1158      metaDescriptor.getColumnFamily(HConstants.TABLE_FAMILY);
1159    final ColumnFamilyDescriptor replBarrierFamilyDesc =
1160      metaDescriptor.getColumnFamily(HConstants.REPLICATION_BARRIER_FAMILY);
1161
1162    this.assignmentManager.joinCluster();
1163    // The below depends on hbase:meta being online.
1164    this.assignmentManager.processOfflineRegions();
1165    // this must be called after the above processOfflineRegions to prevent race
1166    this.assignmentManager.wakeMetaLoadedEvent();
1167
1168    // for migrating from a version without HBASE-25099, and also for honoring the configuration
1169    // first.
1170    if (conf.get(HConstants.META_REPLICAS_NUM) != null) {
1171      int replicasNumInConf =
1172        conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM);
1173      TableDescriptor metaDesc = tableDescriptors.get(TableName.META_TABLE_NAME);
1174      if (metaDesc.getRegionReplication() != replicasNumInConf) {
1175        // it is possible that we already have some replicas before upgrading, so we must set the
1176        // region replication number in meta TableDescriptor directly first, without creating a
1177        // ModifyTableProcedure, otherwise it may cause a double assign for the meta replicas.
1178        int existingReplicasCount =
1179          assignmentManager.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).size();
1180        if (existingReplicasCount > metaDesc.getRegionReplication()) {
1181          LOG.info("Update replica count of hbase:meta from {}(in TableDescriptor)"
1182            + " to {}(existing ZNodes)", metaDesc.getRegionReplication(), existingReplicasCount);
1183          metaDesc = TableDescriptorBuilder.newBuilder(metaDesc)
1184            .setRegionReplication(existingReplicasCount).build();
1185          tableDescriptors.update(metaDesc);
1186        }
1187        // check again, and issue a ModifyTableProcedure if needed
1188        if (metaDesc.getRegionReplication() != replicasNumInConf) {
1189          LOG.info(
1190            "The {} config is {} while the replica count in TableDescriptor is {}"
1191              + " for hbase:meta, altering...",
1192            HConstants.META_REPLICAS_NUM, replicasNumInConf, metaDesc.getRegionReplication());
1193          procedureExecutor.submitProcedure(new ModifyTableProcedure(
1194            procedureExecutor.getEnvironment(), TableDescriptorBuilder.newBuilder(metaDesc)
1195              .setRegionReplication(replicasNumInConf).build(),
1196            null, metaDesc, false, true));
1197        }
1198      }
1199    }
1200    // Initialize after meta is up as below scans meta
1201    FavoredNodesManager fnm = getFavoredNodesManager();
1202    if (fnm != null) {
1203      fnm.initializeFromMeta();
1204    }
1205
1206    // set cluster status again after user regions are assigned
1207    this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
1208
1209    // Start balancer and meta catalog janitor after meta and regions have been assigned.
1210    startupTaskGroup.addTask("Starting balancer and catalog janitor");
1211    this.clusterStatusChore = new ClusterStatusChore(this, balancer);
1212    getChoreService().scheduleChore(clusterStatusChore);
1213    this.balancerChore = new BalancerChore(this);
1214    if (!disableBalancerChoreForTest) {
1215      getChoreService().scheduleChore(balancerChore);
1216    }
1217    if (regionNormalizerManager != null) {
1218      getChoreService().scheduleChore(regionNormalizerManager.getRegionNormalizerChore());
1219    }
1220    this.catalogJanitorChore = new CatalogJanitor(this);
1221    getChoreService().scheduleChore(catalogJanitorChore);
1222    this.hbckChore = new HbckChore(this);
1223    getChoreService().scheduleChore(hbckChore);
1224    this.serverManager.startChore();
1225
1226    // Only for rolling upgrade, where we need to migrate the data in namespace table to meta table.
1227    if (!waitForNamespaceOnline()) {
1228      return;
1229    }
1230    startupTaskGroup.addTask("Starting cluster schema service");
1231    try {
1232      initClusterSchemaService();
1233    } catch (IllegalStateException e) {
1234      if (
1235        e.getCause() != null && e.getCause() instanceof NoSuchColumnFamilyException
1236          && tableFamilyDesc == null && replBarrierFamilyDesc == null
1237      ) {
1238        LOG.info("ClusterSchema service could not be initialized. This is "
1239          + "expected during HBase 1 to 2 upgrade", e);
1240      } else {
1241        throw e;
1242      }
1243    }
1244
1245    if (this.cpHost != null) {
1246      try {
1247        this.cpHost.preMasterInitialization();
1248      } catch (IOException e) {
1249        LOG.error("Coprocessor preMasterInitialization() hook failed", e);
1250      }
1251    }
1252
1253    LOG.info(String.format("Master has completed initialization %.3fsec",
1254      (EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));
1255    this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();
1256    configurationManager.registerObserver(this.balancer);
1257    configurationManager.registerObserver(this.logCleanerPool);
1258    configurationManager.registerObserver(this.logCleaner);
1259    configurationManager.registerObserver(this.regionsRecoveryConfigManager);
1260    configurationManager.registerObserver(this.exclusiveHFileCleanerPool);
1261    if (this.sharedHFileCleanerPool != null) {
1262      configurationManager.registerObserver(this.sharedHFileCleanerPool);
1263    }
1264    if (this.hfileCleaners != null) {
1265      for (HFileCleaner cleaner : hfileCleaners) {
1266        configurationManager.registerObserver(cleaner);
1267      }
1268    }
1269    // Set master as 'initialized'.
1270    setInitialized(true);
1271    startupTaskGroup.markComplete("Initialization successful");
1272    MonitoredTask status =
1273      TaskMonitor.get().createStatus("Progress after master initialized", false, true);
1274
1275    if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
1276      // create missing CFs in meta table after master is set to 'initialized'.
1277      createMissingCFsInMetaDuringUpgrade(metaDescriptor);
1278
1279      // Throwing this Exception to abort active master is painful but this
1280      // seems the only way to add missing CFs in meta while upgrading from
1281      // HBase 1 to 2 (where HBase 2 has HBASE-23055 & HBASE-23782 checked-in).
1282      // So, why do we abort active master after adding missing CFs in meta?
1283      // When we reach here, we would have already bypassed NoSuchColumnFamilyException
1284      // in initClusterSchemaService(), meaning ClusterSchemaService is not
1285      // correctly initialized but we bypassed it. Similarly, we bypassed
1286      // tableStateManager.start() as well. Hence, we should better abort
1287      // current active master because our main task - adding missing CFs
1288      // in meta table is done (possible only after master state is set as
1289      // initialized) at the expense of bypassing few important tasks as part
1290      // of active master init routine. So now we abort active master so that
1291      // next active master init will not face any issues and all mandatory
1292      // services will be started during master init phase.
1293      throw new PleaseRestartMasterException("Aborting active master after missing"
1294        + " CFs are successfully added in meta. Subsequent active master "
1295        + "initialization should be uninterrupted");
1296    }
1297
1298    if (maintenanceMode) {
1299      LOG.info("Detected repair mode, skipping final initialization steps.");
1300      return;
1301    }
1302
1303    assignmentManager.checkIfShouldMoveSystemRegionAsync();
1304    status.setStatus("Starting quota manager");
1305    initQuotaManager();
1306    if (QuotaUtil.isQuotaEnabled(conf)) {
1307      // Create the quota snapshot notifier
1308      spaceQuotaSnapshotNotifier = createQuotaSnapshotNotifier();
1309      spaceQuotaSnapshotNotifier.initialize(getConnection());
1310      this.quotaObserverChore = new QuotaObserverChore(this, getMasterMetrics());
1311      // Start the chore to read the region FS space reports and act on them
1312      getChoreService().scheduleChore(quotaObserverChore);
1313
1314      this.snapshotQuotaChore = new SnapshotQuotaObserverChore(this, getMasterMetrics());
1315      // Start the chore to read snapshots and add their usage to table/NS quotas
1316      getChoreService().scheduleChore(snapshotQuotaChore);
1317    }
1318    final SlowLogMasterService slowLogMasterService = new SlowLogMasterService(conf, this);
1319    slowLogMasterService.init();
1320
1321    WALEventTrackerTableCreator.createIfNeededAndNotExists(conf, this);
1322    // Create REPLICATION.SINK_TRACKER table if needed.
1323    ReplicationSinkTrackerTableCreator.createIfNeededAndNotExists(conf, this);
1324
1325    // clear the dead servers with same host name and port of online server because we are not
1326    // removing dead server with same hostname and port of rs which is trying to check in before
1327    // master initialization. See HBASE-5916.
1328    this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
1329
1330    // Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
1331    status.setStatus("Checking ZNode ACLs");
1332    zooKeeper.checkAndSetZNodeAcls();
1333
1334    status.setStatus("Initializing MOB Cleaner");
1335    initMobCleaner();
1336
1337    // delete the stale data for replication sync up tool if necessary
1338    status.setStatus("Cleanup ReplicationSyncUp status if necessary");
1339    Path replicationSyncUpInfoFile =
1340      new Path(new Path(dataRootDir, ReplicationSyncUp.INFO_DIR), ReplicationSyncUp.INFO_FILE);
1341    if (dataFs.exists(replicationSyncUpInfoFile)) {
1342      // info file is available, load the timestamp and use it to clean up stale data in replication
1343      // queue storage.
1344      byte[] data;
1345      try (FSDataInputStream in = dataFs.open(replicationSyncUpInfoFile)) {
1346        data = ByteStreams.toByteArray(in);
1347      }
1348      ReplicationSyncUpToolInfo info = null;
1349      try {
1350        info = JsonMapper.fromJson(Bytes.toString(data), ReplicationSyncUpToolInfo.class);
1351      } catch (JsonParseException e) {
1352        // usually this should be a partial file, which means the ReplicationSyncUp tool did not
1353        // finish properly, so not a problem. Here we do not clean up the status as we do not know
1354        // the reason why the tool did not finish properly, so let users clean the status up
1355        // manually
1356        LOG.warn("failed to parse replication sync up info file, ignore and continue...", e);
1357      }
1358      if (info != null) {
1359        LOG.info("Remove last sequence ids and hfile references which are written before {}({})",
1360          info.getStartTimeMs(), DateTimeFormatter.ISO_DATE_TIME.withZone(ZoneId.systemDefault())
1361            .format(Instant.ofEpochMilli(info.getStartTimeMs())));
1362        replicationPeerManager.getQueueStorage()
1363          .removeLastSequenceIdsAndHFileRefsBefore(info.getStartTimeMs());
1364        // delete the file after removing the stale data, so next time we do not need to do this
1365        // again.
1366        dataFs.delete(replicationSyncUpInfoFile, false);
1367      }
1368    }
1369    status.setStatus("Calling postStartMaster coprocessors");
1370    if (this.cpHost != null) {
1371      // don't let cp initialization errors kill the master
1372      try {
1373        this.cpHost.postStartMaster();
1374      } catch (IOException ioe) {
1375        LOG.error("Coprocessor postStartMaster() hook failed", ioe);
1376      }
1377    }
1378
1379    zombieDetector.interrupt();
1380
1381    /*
1382     * After master has started up, lets do balancer post startup initialization. Since this runs in
1383     * activeMasterManager thread, it should be fine.
1384     */
1385    long start = EnvironmentEdgeManager.currentTime();
1386    this.balancer.postMasterStartupInitialize();
1387    if (LOG.isDebugEnabled()) {
1388      LOG.debug("Balancer post startup initialization complete, took "
1389        + ((EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds");
1390    }
1391
1392    this.rollingUpgradeChore = new RollingUpgradeChore(this);
1393    getChoreService().scheduleChore(rollingUpgradeChore);
1394
1395    this.oldWALsDirSizeChore = new OldWALsDirSizeChore(this);
1396    getChoreService().scheduleChore(this.oldWALsDirSizeChore);
1397
1398    status.markComplete("Progress after master initialized complete");
1399  }
1400
1401  /**
1402   * Used for testing only to set Mock objects.
1403   * @param hbckChore hbckChore
1404   */
1405  public void setHbckChoreForTesting(HbckChore hbckChore) {
1406    this.hbckChore = hbckChore;
1407  }
1408
1409  /**
1410   * Used for testing only to set Mock objects.
1411   * @param catalogJanitorChore catalogJanitorChore
1412   */
1413  public void setCatalogJanitorChoreForTesting(CatalogJanitor catalogJanitorChore) {
1414    this.catalogJanitorChore = catalogJanitorChore;
1415  }
1416
1417  private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
1418    throws IOException {
1419    TableDescriptor newMetaDesc = TableDescriptorBuilder.newBuilder(metaDescriptor)
1420      .setColumnFamily(FSTableDescriptors.getTableFamilyDescForMeta(conf))
1421      .setColumnFamily(FSTableDescriptors.getReplBarrierFamilyDescForMeta()).build();
1422    long pid = this.modifyTable(TableName.META_TABLE_NAME, () -> newMetaDesc, 0, 0, false);
1423    int tries = 30;
1424    while (
1425      !(getMasterProcedureExecutor().isFinished(pid)) && getMasterProcedureExecutor().isRunning()
1426        && tries > 0
1427    ) {
1428      try {
1429        Thread.sleep(1000);
1430      } catch (InterruptedException e) {
1431        throw new IOException("Wait interrupted", e);
1432      }
1433      tries--;
1434    }
1435    if (tries <= 0) {
1436      throw new HBaseIOException(
1437        "Failed to add table and rep_barrier CFs to meta in a given time.");
1438    } else {
1439      Procedure<?> result = getMasterProcedureExecutor().getResult(pid);
1440      if (result != null && result.isFailed()) {
1441        throw new IOException("Failed to add table and rep_barrier CFs to meta. "
1442          + MasterProcedureUtil.unwrapRemoteIOException(result));
1443      }
1444    }
1445  }
1446
1447  /**
1448   * Check hbase:meta is up and ready for reading. For use during Master startup only.
1449   * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
1450   *         and we will hold here until operator intervention.
1451   */
1452  @InterfaceAudience.Private
1453  public boolean waitForMetaOnline() {
1454    return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
1455  }
1456
1457  /**
1458   * @return True if region is online and scannable else false if an error or shutdown (Otherwise we
1459   *         just block in here holding up all forward-progess).
1460   */
1461  private boolean isRegionOnline(RegionInfo ri) {
1462    RetryCounter rc = null;
1463    while (!isStopped()) {
1464      RegionState rs = this.assignmentManager.getRegionStates().getRegionState(ri);
1465      if (rs != null && rs.isOpened()) {
1466        if (this.getServerManager().isServerOnline(rs.getServerName())) {
1467          return true;
1468        }
1469      }
1470      // Region is not OPEN.
1471      Optional<Procedure<MasterProcedureEnv>> optProc = this.procedureExecutor.getProcedures()
1472        .stream().filter(p -> p instanceof ServerCrashProcedure).findAny();
1473      // TODO: Add a page to refguide on how to do repair. Have this log message point to it.
1474      // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
1475      // then how to assign including how to break region lock if one held.
1476      LOG.warn(
1477        "{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot "
1478          + "progress, in holding-pattern until region onlined.",
1479        ri.getRegionNameAsString(), rs, optProc.isPresent());
1480      // Check once-a-minute.
1481      if (rc == null) {
1482        rc = new RetryCounterFactory(Integer.MAX_VALUE, 1000, 60_000).create();
1483      }
1484      Threads.sleep(rc.getBackoffTimeAndIncrementAttempts());
1485    }
1486    return false;
1487  }
1488
1489  /**
1490   * Check hbase:namespace table is assigned. If not, startup will hang looking for the ns table
1491   * <p/>
1492   * This is for rolling upgrading, later we will migrate the data in ns table to the ns family of
1493   * meta table. And if this is a new cluster, this method will return immediately as there will be
1494   * no namespace table/region.
1495   * @return True if namespace table is up/online.
1496   */
1497  private boolean waitForNamespaceOnline() throws IOException {
1498    TableState nsTableState =
1499      MetaTableAccessor.getTableState(getConnection(), TableName.NAMESPACE_TABLE_NAME);
1500    if (nsTableState == null || nsTableState.isDisabled()) {
1501      // this means we have already migrated the data and disabled or deleted the namespace table,
1502      // or this is a new deploy which does not have a namespace table from the beginning.
1503      return true;
1504    }
1505    List<RegionInfo> ris =
1506      this.assignmentManager.getRegionStates().getRegionsOfTable(TableName.NAMESPACE_TABLE_NAME);
1507    if (ris.isEmpty()) {
1508      // maybe this will not happen any more, but anyway, no harm to add a check here...
1509      return true;
1510    }
1511    // Else there are namespace regions up in meta. Ensure they are assigned before we go on.
1512    for (RegionInfo ri : ris) {
1513      if (!isRegionOnline(ri)) {
1514        return false;
1515      }
1516    }
1517    return true;
1518  }
1519
1520  /**
1521   * Adds the {@code MasterQuotasObserver} to the list of configured Master observers to
1522   * automatically remove quotas for a table when that table is deleted.
1523   */
1524  @InterfaceAudience.Private
1525  public void updateConfigurationForQuotasObserver(Configuration conf) {
1526    // We're configured to not delete quotas on table deletion, so we don't need to add the obs.
1527    if (
1528      !conf.getBoolean(MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE,
1529        MasterQuotasObserver.REMOVE_QUOTA_ON_TABLE_DELETE_DEFAULT)
1530    ) {
1531      return;
1532    }
1533    String[] masterCoprocs = conf.getStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY);
1534    final int length = null == masterCoprocs ? 0 : masterCoprocs.length;
1535    String[] updatedCoprocs = new String[length + 1];
1536    if (length > 0) {
1537      System.arraycopy(masterCoprocs, 0, updatedCoprocs, 0, masterCoprocs.length);
1538    }
1539    updatedCoprocs[length] = MasterQuotasObserver.class.getName();
1540    conf.setStrings(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, updatedCoprocs);
1541  }
1542
1543  private void initMobCleaner() {
1544    this.mobFileCleanerChore = new MobFileCleanerChore(this);
1545    getChoreService().scheduleChore(mobFileCleanerChore);
1546    this.mobFileCompactionChore = new MobFileCompactionChore(this);
1547    getChoreService().scheduleChore(mobFileCompactionChore);
1548  }
1549
1550  /**
1551   * <p>
1552   * Create a {@link ServerManager} instance.
1553   * </p>
1554   * <p>
1555   * Will be overridden in tests.
1556   * </p>
1557   */
1558  @InterfaceAudience.Private
1559  protected ServerManager createServerManager(MasterServices master, RegionServerList storage)
1560    throws IOException {
1561    // We put this out here in a method so can do a Mockito.spy and stub it out
1562    // w/ a mocked up ServerManager.
1563    setupClusterConnection();
1564    return new ServerManager(master, storage);
1565  }
1566
1567  private void waitForRegionServers(final MonitoredTask status)
1568    throws IOException, InterruptedException {
1569    this.serverManager.waitForRegionServers(status);
1570  }
1571
1572  // Will be overridden in tests
1573  @InterfaceAudience.Private
1574  protected void initClusterSchemaService() throws IOException, InterruptedException {
1575    this.clusterSchemaService = new ClusterSchemaServiceImpl(this);
1576    this.clusterSchemaService.startAsync();
1577    try {
1578      this.clusterSchemaService
1579        .awaitRunning(getConfiguration().getInt(HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS,
1580          DEFAULT_HBASE_MASTER_WAIT_ON_SERVICE_IN_SECONDS), TimeUnit.SECONDS);
1581    } catch (TimeoutException toe) {
1582      throw new IOException("Timedout starting ClusterSchemaService", toe);
1583    }
1584  }
1585
1586  private void initQuotaManager() throws IOException {
1587    MasterQuotaManager quotaManager = new MasterQuotaManager(this);
1588    quotaManager.start();
1589    this.quotaManager = quotaManager;
1590  }
1591
1592  private SpaceQuotaSnapshotNotifier createQuotaSnapshotNotifier() {
1593    SpaceQuotaSnapshotNotifier notifier =
1594      SpaceQuotaSnapshotNotifierFactory.getInstance().create(getConfiguration());
1595    return notifier;
1596  }
1597
1598  public boolean isCatalogJanitorEnabled() {
1599    return catalogJanitorChore != null ? catalogJanitorChore.getEnabled() : false;
1600  }
1601
1602  boolean isCleanerChoreEnabled() {
1603    boolean hfileCleanerFlag = true, logCleanerFlag = true;
1604
1605    if (getHFileCleaner() != null) {
1606      hfileCleanerFlag = getHFileCleaner().getEnabled();
1607    }
1608
1609    if (logCleaner != null) {
1610      logCleanerFlag = logCleaner.getEnabled();
1611    }
1612
1613    return (hfileCleanerFlag && logCleanerFlag);
1614  }
1615
1616  @Override
1617  public ServerManager getServerManager() {
1618    return this.serverManager;
1619  }
1620
1621  @Override
1622  public MasterFileSystem getMasterFileSystem() {
1623    return this.fileSystemManager;
1624  }
1625
1626  @Override
1627  public MasterWalManager getMasterWalManager() {
1628    return this.walManager;
1629  }
1630
1631  @Override
1632  public SplitWALManager getSplitWALManager() {
1633    return splitWALManager;
1634  }
1635
1636  @Override
1637  public TableStateManager getTableStateManager() {
1638    return tableStateManager;
1639  }
1640
1641  /*
1642   * Start up all services. If any of these threads gets an unhandled exception then they just die
1643   * with a logged message. This should be fine because in general, we do not expect the master to
1644   * get such unhandled exceptions as OOMEs; it should be lightly loaded. See what HRegionServer
1645   * does if need to install an unexpected exception handler.
1646   */
1647  private void startServiceThreads() throws IOException {
1648    // Start the executor service pools
1649    final int masterOpenRegionPoolSize = conf.getInt(HConstants.MASTER_OPEN_REGION_THREADS,
1650      HConstants.MASTER_OPEN_REGION_THREADS_DEFAULT);
1651    executorService.startExecutorService(executorService.new ExecutorConfig()
1652      .setExecutorType(ExecutorType.MASTER_OPEN_REGION).setCorePoolSize(masterOpenRegionPoolSize));
1653    final int masterCloseRegionPoolSize = conf.getInt(HConstants.MASTER_CLOSE_REGION_THREADS,
1654      HConstants.MASTER_CLOSE_REGION_THREADS_DEFAULT);
1655    executorService.startExecutorService(
1656      executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_CLOSE_REGION)
1657        .setCorePoolSize(masterCloseRegionPoolSize));
1658    final int masterServerOpThreads = conf.getInt(HConstants.MASTER_SERVER_OPERATIONS_THREADS,
1659      HConstants.MASTER_SERVER_OPERATIONS_THREADS_DEFAULT);
1660    executorService.startExecutorService(
1661      executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_SERVER_OPERATIONS)
1662        .setCorePoolSize(masterServerOpThreads));
1663    final int masterServerMetaOpsThreads =
1664      conf.getInt(HConstants.MASTER_META_SERVER_OPERATIONS_THREADS,
1665        HConstants.MASTER_META_SERVER_OPERATIONS_THREADS_DEFAULT);
1666    executorService.startExecutorService(executorService.new ExecutorConfig()
1667      .setExecutorType(ExecutorType.MASTER_META_SERVER_OPERATIONS)
1668      .setCorePoolSize(masterServerMetaOpsThreads));
1669    final int masterLogReplayThreads = conf.getInt(HConstants.MASTER_LOG_REPLAY_OPS_THREADS,
1670      HConstants.MASTER_LOG_REPLAY_OPS_THREADS_DEFAULT);
1671    executorService.startExecutorService(executorService.new ExecutorConfig()
1672      .setExecutorType(ExecutorType.M_LOG_REPLAY_OPS).setCorePoolSize(masterLogReplayThreads));
1673    final int masterSnapshotThreads = conf.getInt(SnapshotManager.SNAPSHOT_POOL_THREADS_KEY,
1674      SnapshotManager.SNAPSHOT_POOL_THREADS_DEFAULT);
1675    executorService.startExecutorService(
1676      executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_SNAPSHOT_OPERATIONS)
1677        .setCorePoolSize(masterSnapshotThreads).setAllowCoreThreadTimeout(true));
1678    final int masterMergeDispatchThreads = conf.getInt(HConstants.MASTER_MERGE_DISPATCH_THREADS,
1679      HConstants.MASTER_MERGE_DISPATCH_THREADS_DEFAULT);
1680    executorService.startExecutorService(
1681      executorService.new ExecutorConfig().setExecutorType(ExecutorType.MASTER_MERGE_OPERATIONS)
1682        .setCorePoolSize(masterMergeDispatchThreads).setAllowCoreThreadTimeout(true));
1683
1684    // We depend on there being only one instance of this executor running
1685    // at a time. To do concurrency, would need fencing of enable/disable of
1686    // tables.
1687    // Any time changing this maxThreads to > 1, pls see the comment at
1688    // AccessController#postCompletedCreateTableAction
1689    executorService.startExecutorService(executorService.new ExecutorConfig()
1690      .setExecutorType(ExecutorType.MASTER_TABLE_OPERATIONS).setCorePoolSize(1));
1691    startProcedureExecutor();
1692
1693    // Create log cleaner thread pool
1694    logCleanerPool = DirScanPool.getLogCleanerScanPool(conf);
1695    Map<String, Object> params = new HashMap<>();
1696    params.put(MASTER, this);
1697    // Start log cleaner thread
1698    int cleanerInterval =
1699      conf.getInt(HBASE_MASTER_CLEANER_INTERVAL, DEFAULT_HBASE_MASTER_CLEANER_INTERVAL);
1700    this.logCleaner =
1701      new LogCleaner(cleanerInterval, this, conf, getMasterWalManager().getFileSystem(),
1702        getMasterWalManager().getOldLogDir(), logCleanerPool, params);
1703    getChoreService().scheduleChore(logCleaner);
1704
1705    Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
1706
1707    // Create custom archive hfile cleaners
1708    String[] paths = conf.getStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS);
1709    // todo: handle the overlap issues for the custom paths
1710
1711    if (paths != null && paths.length > 0) {
1712      if (conf.getStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS) == null) {
1713        Set<String> cleanerClasses = new HashSet<>();
1714        String[] cleaners = conf.getStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
1715        if (cleaners != null) {
1716          Collections.addAll(cleanerClasses, cleaners);
1717        }
1718        conf.setStrings(HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS,
1719          cleanerClasses.toArray(new String[cleanerClasses.size()]));
1720        LOG.info("Archive custom cleaner paths: {}, plugins: {}", Arrays.asList(paths),
1721          cleanerClasses);
1722      }
1723      // share the hfile cleaner pool in custom paths
1724      sharedHFileCleanerPool = DirScanPool.getHFileCleanerScanPool(conf.get(CUSTOM_POOL_SIZE, "6"));
1725      for (int i = 0; i < paths.length; i++) {
1726        Path path = new Path(paths[i].trim());
1727        HFileCleaner cleaner =
1728          new HFileCleaner("ArchiveCustomHFileCleaner-" + path.getName(), cleanerInterval, this,
1729            conf, getMasterFileSystem().getFileSystem(), new Path(archiveDir, path),
1730            HFileCleaner.HFILE_CLEANER_CUSTOM_PATHS_PLUGINS, sharedHFileCleanerPool, params, null);
1731        hfileCleaners.add(cleaner);
1732        hfileCleanerPaths.add(path);
1733      }
1734    }
1735
1736    // Create the whole archive dir cleaner thread pool
1737    exclusiveHFileCleanerPool = DirScanPool.getHFileCleanerScanPool(conf);
1738    hfileCleaners.add(0,
1739      new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem().getFileSystem(),
1740        archiveDir, exclusiveHFileCleanerPool, params, hfileCleanerPaths));
1741    hfileCleanerPaths.add(0, archiveDir);
1742    // Schedule all the hfile cleaners
1743    for (HFileCleaner hFileCleaner : hfileCleaners) {
1744      getChoreService().scheduleChore(hFileCleaner);
1745    }
1746
1747    // Regions Reopen based on very high storeFileRefCount is considered enabled
1748    // only if hbase.regions.recovery.store.file.ref.count has value > 0
1749    final int maxStoreFileRefCount = conf.getInt(HConstants.STORE_FILE_REF_COUNT_THRESHOLD,
1750      HConstants.DEFAULT_STORE_FILE_REF_COUNT_THRESHOLD);
1751    if (maxStoreFileRefCount > 0) {
1752      this.regionsRecoveryChore = new RegionsRecoveryChore(this, conf, this);
1753      getChoreService().scheduleChore(this.regionsRecoveryChore);
1754    } else {
1755      LOG.info(
1756        "Reopening regions with very high storeFileRefCount is disabled. "
1757          + "Provide threshold value > 0 for {} to enable it.",
1758        HConstants.STORE_FILE_REF_COUNT_THRESHOLD);
1759    }
1760
1761    this.regionsRecoveryConfigManager = new RegionsRecoveryConfigManager(this);
1762
1763    replicationBarrierCleaner =
1764      new ReplicationBarrierCleaner(conf, this, getConnection(), replicationPeerManager);
1765    getChoreService().scheduleChore(replicationBarrierCleaner);
1766
1767    final boolean isSnapshotChoreEnabled = this.snapshotCleanupStateStore.get();
1768    this.snapshotCleanerChore = new SnapshotCleanerChore(this, conf, getSnapshotManager());
1769    if (isSnapshotChoreEnabled) {
1770      getChoreService().scheduleChore(this.snapshotCleanerChore);
1771    } else {
1772      if (LOG.isTraceEnabled()) {
1773        LOG.trace("Snapshot Cleaner Chore is disabled. Not starting up the chore..");
1774      }
1775    }
1776    serviceStarted = true;
1777    if (LOG.isTraceEnabled()) {
1778      LOG.trace("Started service threads");
1779    }
1780  }
1781
1782  protected void stopServiceThreads() {
1783    if (masterJettyServer != null) {
1784      LOG.info("Stopping master jetty server");
1785      try {
1786        masterJettyServer.stop();
1787      } catch (Exception e) {
1788        LOG.error("Failed to stop master jetty server", e);
1789      }
1790    }
1791    stopChoreService();
1792    stopExecutorService();
1793    if (exclusiveHFileCleanerPool != null) {
1794      exclusiveHFileCleanerPool.shutdownNow();
1795      exclusiveHFileCleanerPool = null;
1796    }
1797    if (logCleanerPool != null) {
1798      logCleanerPool.shutdownNow();
1799      logCleanerPool = null;
1800    }
1801    if (sharedHFileCleanerPool != null) {
1802      sharedHFileCleanerPool.shutdownNow();
1803      sharedHFileCleanerPool = null;
1804    }
1805    if (maintenanceRegionServer != null) {
1806      maintenanceRegionServer.getRegionServer().stop(HBASE_MASTER_CLEANER_INTERVAL);
1807    }
1808
1809    LOG.debug("Stopping service threads");
1810    // stop procedure executor prior to other services such as server manager and assignment
1811    // manager, as these services are important for some running procedures. See HBASE-24117 for
1812    // example.
1813    stopProcedureExecutor();
1814
1815    if (regionNormalizerManager != null) {
1816      regionNormalizerManager.stop();
1817    }
1818    if (this.quotaManager != null) {
1819      this.quotaManager.stop();
1820    }
1821
1822    if (this.activeMasterManager != null) {
1823      this.activeMasterManager.stop();
1824    }
1825    if (this.serverManager != null) {
1826      this.serverManager.stop();
1827    }
1828    if (this.assignmentManager != null) {
1829      this.assignmentManager.stop();
1830    }
1831
1832    if (masterRegion != null) {
1833      masterRegion.close(isAborted());
1834    }
1835    if (this.walManager != null) {
1836      this.walManager.stop();
1837    }
1838    if (this.fileSystemManager != null) {
1839      this.fileSystemManager.stop();
1840    }
1841    if (this.mpmHost != null) {
1842      this.mpmHost.stop("server shutting down.");
1843    }
1844    if (this.regionServerTracker != null) {
1845      this.regionServerTracker.stop();
1846    }
1847  }
1848
1849  private void createProcedureExecutor() throws IOException {
1850    final String procedureDispatcherClassName =
1851      conf.get(HBASE_MASTER_RSPROC_DISPATCHER_CLASS, DEFAULT_HBASE_MASTER_RSPROC_DISPATCHER_CLASS);
1852    final RSProcedureDispatcher procedureDispatcher = ReflectionUtils.instantiateWithCustomCtor(
1853      procedureDispatcherClassName, new Class[] { MasterServices.class }, new Object[] { this });
1854    final MasterProcedureEnv procEnv = new MasterProcedureEnv(this, procedureDispatcher);
1855    procedureStore = new RegionProcedureStore(this, masterRegion,
1856      new MasterProcedureEnv.FsUtilsLeaseRecovery(this));
1857    procedureStore.registerListener(new ProcedureStoreListener() {
1858
1859      @Override
1860      public void abortProcess() {
1861        abort("The Procedure Store lost the lease", null);
1862      }
1863    });
1864    MasterProcedureScheduler procedureScheduler = procEnv.getProcedureScheduler();
1865    procedureExecutor = new ProcedureExecutor<>(conf, procEnv, procedureStore, procedureScheduler);
1866    configurationManager.registerObserver(procEnv);
1867
1868    int cpus = Runtime.getRuntime().availableProcessors();
1869    final int numThreads = conf.getInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, Math.max(
1870      (cpus > 0 ? cpus / 4 : 0), MasterProcedureConstants.DEFAULT_MIN_MASTER_PROCEDURE_THREADS));
1871    final boolean abortOnCorruption =
1872      conf.getBoolean(MasterProcedureConstants.EXECUTOR_ABORT_ON_CORRUPTION,
1873        MasterProcedureConstants.DEFAULT_EXECUTOR_ABORT_ON_CORRUPTION);
1874    procedureStore.start(numThreads);
1875    // Just initialize it but do not start the workers, we will start the workers later by calling
1876    // startProcedureExecutor. See the javadoc for finishActiveMasterInitialization for more
1877    // details.
1878    procedureExecutor.init(numThreads, abortOnCorruption);
1879    if (!procEnv.getRemoteDispatcher().start()) {
1880      throw new HBaseIOException("Failed start of remote dispatcher");
1881    }
1882  }
1883
1884  // will be override in UT
1885  protected void startProcedureExecutor() throws IOException {
1886    procedureExecutor.startWorkers();
1887  }
1888
1889  /**
1890   * Turn on/off Snapshot Cleanup Chore
1891   * @param on indicates whether Snapshot Cleanup Chore is to be run
1892   */
1893  void switchSnapshotCleanup(final boolean on, final boolean synchronous) throws IOException {
1894    if (synchronous) {
1895      synchronized (this.snapshotCleanerChore) {
1896        switchSnapshotCleanup(on);
1897      }
1898    } else {
1899      switchSnapshotCleanup(on);
1900    }
1901  }
1902
1903  private void switchSnapshotCleanup(final boolean on) throws IOException {
1904    snapshotCleanupStateStore.set(on);
1905    if (on) {
1906      getChoreService().scheduleChore(this.snapshotCleanerChore);
1907    } else {
1908      this.snapshotCleanerChore.cancel();
1909    }
1910  }
1911
1912  private void stopProcedureExecutor() {
1913    if (procedureExecutor != null) {
1914      configurationManager.deregisterObserver(procedureExecutor.getEnvironment());
1915      procedureExecutor.getEnvironment().getRemoteDispatcher().stop();
1916      procedureExecutor.stop();
1917      procedureExecutor.join();
1918      procedureExecutor = null;
1919    }
1920
1921    if (procedureStore != null) {
1922      procedureStore.stop(isAborted());
1923      procedureStore = null;
1924    }
1925  }
1926
1927  protected void stopChores() {
1928    shutdownChore(mobFileCleanerChore);
1929    shutdownChore(mobFileCompactionChore);
1930    shutdownChore(balancerChore);
1931    if (regionNormalizerManager != null) {
1932      shutdownChore(regionNormalizerManager.getRegionNormalizerChore());
1933    }
1934    shutdownChore(clusterStatusChore);
1935    shutdownChore(catalogJanitorChore);
1936    shutdownChore(clusterStatusPublisherChore);
1937    shutdownChore(snapshotQuotaChore);
1938    shutdownChore(logCleaner);
1939    if (hfileCleaners != null) {
1940      for (ScheduledChore chore : hfileCleaners) {
1941        chore.shutdown();
1942      }
1943      hfileCleaners = null;
1944    }
1945    shutdownChore(replicationBarrierCleaner);
1946    shutdownChore(snapshotCleanerChore);
1947    shutdownChore(hbckChore);
1948    shutdownChore(regionsRecoveryChore);
1949    shutdownChore(rollingUpgradeChore);
1950    shutdownChore(oldWALsDirSizeChore);
1951  }
1952
1953  /** Returns Get remote side's InetAddress */
1954  InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
1955    throws UnknownHostException {
1956    // Do it out here in its own little method so can fake an address when
1957    // mocking up in tests.
1958    InetAddress ia = RpcServer.getRemoteIp();
1959
1960    // The call could be from the local regionserver,
1961    // in which case, there is no remote address.
1962    if (ia == null && serverStartCode == startcode) {
1963      InetSocketAddress isa = rpcServices.getSocketAddress();
1964      if (isa != null && isa.getPort() == port) {
1965        ia = isa.getAddress();
1966      }
1967    }
1968    return ia;
1969  }
1970
1971  /** Returns Maximum time we should run balancer for */
1972  private int getMaxBalancingTime() {
1973    // if max balancing time isn't set, defaulting it to period time
1974    int maxBalancingTime =
1975      getConfiguration().getInt(HConstants.HBASE_BALANCER_MAX_BALANCING, getConfiguration()
1976        .getInt(HConstants.HBASE_BALANCER_PERIOD, HConstants.DEFAULT_HBASE_BALANCER_PERIOD));
1977    return maxBalancingTime;
1978  }
1979
1980  /** Returns Maximum number of regions in transition */
1981  private int getMaxRegionsInTransition() {
1982    int numRegions = this.assignmentManager.getRegionStates().getRegionAssignments().size();
1983    return Math.max((int) Math.floor(numRegions * this.maxRitPercent), 1);
1984  }
1985
1986  /**
1987   * It first sleep to the next balance plan start time. Meanwhile, throttling by the max number
1988   * regions in transition to protect availability.
1989   * @param nextBalanceStartTime   The next balance plan start time
1990   * @param maxRegionsInTransition max number of regions in transition
1991   * @param cutoffTime             when to exit balancer
1992   */
1993  private void balanceThrottling(long nextBalanceStartTime, int maxRegionsInTransition,
1994    long cutoffTime) {
1995    boolean interrupted = false;
1996
1997    // Sleep to next balance plan start time
1998    // But if there are zero regions in transition, it can skip sleep to speed up.
1999    while (
2000      !interrupted && EnvironmentEdgeManager.currentTime() < nextBalanceStartTime
2001        && this.assignmentManager.getRegionStates().hasRegionsInTransition()
2002    ) {
2003      try {
2004        Thread.sleep(100);
2005      } catch (InterruptedException ie) {
2006        interrupted = true;
2007      }
2008    }
2009
2010    // Throttling by max number regions in transition
2011    while (
2012      !interrupted && maxRegionsInTransition > 0
2013        && this.assignmentManager.getRegionStates().getRegionsInTransitionCount()
2014            >= maxRegionsInTransition
2015        && EnvironmentEdgeManager.currentTime() <= cutoffTime
2016    ) {
2017      try {
2018        // sleep if the number of regions in transition exceeds the limit
2019        Thread.sleep(100);
2020      } catch (InterruptedException ie) {
2021        interrupted = true;
2022      }
2023    }
2024
2025    if (interrupted) Thread.currentThread().interrupt();
2026  }
2027
2028  public BalanceResponse balance() throws IOException {
2029    return balance(BalanceRequest.defaultInstance());
2030  }
2031
2032  /**
2033   * Trigger a normal balance, see {@link HMaster#balance()} . If the balance is not executed this
2034   * time, the metrics related to the balance will be updated. When balance is running, related
2035   * metrics will be updated at the same time. But if some checking logic failed and cause the
2036   * balancer exit early, we lost the chance to update balancer metrics. This will lead to user
2037   * missing the latest balancer info.
2038   */
2039  public BalanceResponse balanceOrUpdateMetrics() throws IOException {
2040    synchronized (this.balancer) {
2041      BalanceResponse response = balance();
2042      if (!response.isBalancerRan()) {
2043        Map<TableName, Map<ServerName, List<RegionInfo>>> assignments =
2044          this.assignmentManager.getRegionStates().getAssignmentsForBalancer(this.tableStateManager,
2045            this.serverManager.getOnlineServersList());
2046        for (Map<ServerName, List<RegionInfo>> serverMap : assignments.values()) {
2047          serverMap.keySet().removeAll(this.serverManager.getDrainingServersList());
2048        }
2049        this.balancer.updateBalancerLoadInfo(assignments);
2050      }
2051      return response;
2052    }
2053  }
2054
2055  /**
2056   * Checks master state before initiating action over region topology.
2057   * @param action the name of the action under consideration, for logging.
2058   * @return {@code true} when the caller should exit early, {@code false} otherwise.
2059   */
2060  @Override
2061  public boolean skipRegionManagementAction(final String action) {
2062    // Note: this method could be `default` on MasterServices if but for logging.
2063    if (!isInitialized()) {
2064      LOG.debug("Master has not been initialized, don't run {}.", action);
2065      return true;
2066    }
2067    if (this.getServerManager().isClusterShutdown()) {
2068      LOG.info("Cluster is shutting down, don't run {}.", action);
2069      return true;
2070    }
2071    if (isInMaintenanceMode()) {
2072      LOG.info("Master is in maintenance mode, don't run {}.", action);
2073      return true;
2074    }
2075    return false;
2076  }
2077
2078  public BalanceResponse balance(BalanceRequest request) throws IOException {
2079    checkInitialized();
2080
2081    BalanceResponse.Builder responseBuilder = BalanceResponse.newBuilder();
2082
2083    if (loadBalancerStateStore == null || !(loadBalancerStateStore.get() || request.isDryRun())) {
2084      return responseBuilder.build();
2085    }
2086
2087    if (skipRegionManagementAction("balancer")) {
2088      return responseBuilder.build();
2089    }
2090
2091    synchronized (this.balancer) {
2092      // Only allow one balance run at at time.
2093      if (this.assignmentManager.hasRegionsInTransition()) {
2094        List<RegionStateNode> regionsInTransition = assignmentManager.getRegionsInTransition();
2095        // if hbase:meta region is in transition, result of assignment cannot be recorded
2096        // ignore the force flag in that case
2097        boolean metaInTransition = assignmentManager.isMetaRegionInTransition();
2098        List<RegionStateNode> toPrint = regionsInTransition;
2099        int max = 5;
2100        boolean truncated = false;
2101        if (regionsInTransition.size() > max) {
2102          toPrint = regionsInTransition.subList(0, max);
2103          truncated = true;
2104        }
2105
2106        if (!request.isIgnoreRegionsInTransition() || metaInTransition) {
2107          LOG.info("Not running balancer (ignoreRIT=false" + ", metaRIT=" + metaInTransition
2108            + ") because " + regionsInTransition.size() + " region(s) in transition: " + toPrint
2109            + (truncated ? "(truncated list)" : ""));
2110          return responseBuilder.build();
2111        }
2112      }
2113      if (this.serverManager.areDeadServersInProgress()) {
2114        LOG.info("Not running balancer because processing dead regionserver(s): "
2115          + this.serverManager.getDeadServers());
2116        return responseBuilder.build();
2117      }
2118
2119      if (this.cpHost != null) {
2120        try {
2121          if (this.cpHost.preBalance(request)) {
2122            LOG.debug("Coprocessor bypassing balancer request");
2123            return responseBuilder.build();
2124          }
2125        } catch (IOException ioe) {
2126          LOG.error("Error invoking master coprocessor preBalance()", ioe);
2127          return responseBuilder.build();
2128        }
2129      }
2130
2131      Map<TableName, Map<ServerName, List<RegionInfo>>> assignments =
2132        this.assignmentManager.getRegionStates().getAssignmentsForBalancer(tableStateManager,
2133          this.serverManager.getOnlineServersList());
2134      for (Map<ServerName, List<RegionInfo>> serverMap : assignments.values()) {
2135        serverMap.keySet().removeAll(this.serverManager.getDrainingServersList());
2136      }
2137
2138      // Give the balancer the current cluster state.
2139      this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
2140
2141      List<RegionPlan> plans = this.balancer.balanceCluster(assignments);
2142
2143      responseBuilder.setBalancerRan(true).setMovesCalculated(plans == null ? 0 : plans.size());
2144
2145      if (skipRegionManagementAction("balancer")) {
2146        // make one last check that the cluster isn't shutting down before proceeding.
2147        return responseBuilder.build();
2148      }
2149
2150      // For dry run we don't actually want to execute the moves, but we do want
2151      // to execute the coprocessor below
2152      List<RegionPlan> sucRPs =
2153        request.isDryRun() ? Collections.emptyList() : executeRegionPlansWithThrottling(plans);
2154
2155      if (this.cpHost != null) {
2156        try {
2157          this.cpHost.postBalance(request, sucRPs);
2158        } catch (IOException ioe) {
2159          // balancing already succeeded so don't change the result
2160          LOG.error("Error invoking master coprocessor postBalance()", ioe);
2161        }
2162      }
2163
2164      responseBuilder.setMovesExecuted(sucRPs.size());
2165    }
2166
2167    // If LoadBalancer did not generate any plans, it means the cluster is already balanced.
2168    // Return true indicating a success.
2169    return responseBuilder.build();
2170  }
2171
2172  /**
2173   * Execute region plans with throttling
2174   * @param plans to execute
2175   * @return succeeded plans
2176   */
2177  public List<RegionPlan> executeRegionPlansWithThrottling(List<RegionPlan> plans) {
2178    List<RegionPlan> successRegionPlans = new ArrayList<>();
2179    int maxRegionsInTransition = getMaxRegionsInTransition();
2180    long balanceStartTime = EnvironmentEdgeManager.currentTime();
2181    long cutoffTime = balanceStartTime + this.maxBalancingTime;
2182    int rpCount = 0; // number of RegionPlans balanced so far
2183    if (plans != null && !plans.isEmpty()) {
2184      int balanceInterval = this.maxBalancingTime / plans.size();
2185      LOG.info(
2186        "Balancer plans size is " + plans.size() + ", the balance interval is " + balanceInterval
2187          + " ms, and the max number regions in transition is " + maxRegionsInTransition);
2188
2189      for (RegionPlan plan : plans) {
2190        LOG.info("balance " + plan);
2191        // TODO: bulk assign
2192        try {
2193          this.assignmentManager.balance(plan);
2194          this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
2195          this.balancer.throttle(plan);
2196        } catch (HBaseIOException hioe) {
2197          // should ignore failed plans here, avoiding the whole balance plans be aborted
2198          // later calls of balance() can fetch up the failed and skipped plans
2199          LOG.warn("Failed balance plan {}, skipping...", plan, hioe);
2200        } catch (Exception e) {
2201          LOG.warn("Failed throttling assigning a new plan.", e);
2202        }
2203        // rpCount records balance plans processed, does not care if a plan succeeds
2204        rpCount++;
2205        successRegionPlans.add(plan);
2206
2207        if (this.maxBalancingTime > 0) {
2208          balanceThrottling(balanceStartTime + rpCount * balanceInterval, maxRegionsInTransition,
2209            cutoffTime);
2210        }
2211
2212        // if performing next balance exceeds cutoff time, exit the loop
2213        if (
2214          this.maxBalancingTime > 0 && rpCount < plans.size()
2215            && EnvironmentEdgeManager.currentTime() > cutoffTime
2216        ) {
2217          // TODO: After balance, there should not be a cutoff time (keeping it as
2218          // a security net for now)
2219          LOG.debug(
2220            "No more balancing till next balance run; maxBalanceTime=" + this.maxBalancingTime);
2221          break;
2222        }
2223      }
2224    }
2225    LOG.debug("Balancer is going into sleep until next period in {}ms", getConfiguration()
2226      .getInt(HConstants.HBASE_BALANCER_PERIOD, HConstants.DEFAULT_HBASE_BALANCER_PERIOD));
2227    return successRegionPlans;
2228  }
2229
2230  @Override
2231  public RegionNormalizerManager getRegionNormalizerManager() {
2232    return regionNormalizerManager;
2233  }
2234
2235  @Override
2236  public boolean normalizeRegions(final NormalizeTableFilterParams ntfp,
2237    final boolean isHighPriority) throws IOException {
2238    if (regionNormalizerManager == null || !regionNormalizerManager.isNormalizerOn()) {
2239      LOG.debug("Region normalization is disabled, don't run region normalizer.");
2240      return false;
2241    }
2242    if (skipRegionManagementAction("region normalizer")) {
2243      return false;
2244    }
2245    if (assignmentManager.hasRegionsInTransition()) {
2246      return false;
2247    }
2248
2249    final Set<TableName> matchingTables = getTableDescriptors(new LinkedList<>(),
2250      ntfp.getNamespace(), ntfp.getRegex(), ntfp.getTableNames(), false).stream()
2251        .map(TableDescriptor::getTableName).collect(Collectors.toSet());
2252    final Set<TableName> allEnabledTables =
2253      tableStateManager.getTablesInStates(TableState.State.ENABLED);
2254    final List<TableName> targetTables =
2255      new ArrayList<>(Sets.intersection(matchingTables, allEnabledTables));
2256    Collections.shuffle(targetTables);
2257    return regionNormalizerManager.normalizeRegions(targetTables, isHighPriority);
2258  }
2259
2260  /** Returns Client info for use as prefix on an audit log string; who did an action */
2261  @Override
2262  public String getClientIdAuditPrefix() {
2263    return "Client=" + RpcServer.getRequestUserName().orElse(null) + "/"
2264      + RpcServer.getRemoteAddress().orElse(null);
2265  }
2266
2267  /**
2268   * Switch for the background CatalogJanitor thread. Used for testing. The thread will continue to
2269   * run. It will just be a noop if disabled.
2270   * @param b If false, the catalog janitor won't do anything.
2271   */
2272  public void setCatalogJanitorEnabled(final boolean b) {
2273    this.catalogJanitorChore.setEnabled(b);
2274  }
2275
2276  @Override
2277  public long mergeRegions(final RegionInfo[] regionsToMerge, final boolean forcible, final long ng,
2278    final long nonce) throws IOException {
2279    checkInitialized();
2280
2281    final String regionNamesToLog = RegionInfo.getShortNameToLog(regionsToMerge);
2282
2283    if (!isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
2284      LOG.warn("Merge switch is off! skip merge of " + regionNamesToLog);
2285      throw new DoNotRetryIOException(
2286        "Merge of " + regionNamesToLog + " failed because merge switch is off");
2287    }
2288
2289    if (!getTableDescriptors().get(regionsToMerge[0].getTable()).isMergeEnabled()) {
2290      LOG.warn("Merge is disabled for the table! Skipping merge of {}", regionNamesToLog);
2291      throw new DoNotRetryIOException(
2292        "Merge of " + regionNamesToLog + " failed as region merge is disabled for the table");
2293    }
2294
2295    return MasterProcedureUtil.submitProcedure(new NonceProcedureRunnable(this, ng, nonce) {
2296      @Override
2297      protected void run() throws IOException {
2298        getMaster().getMasterCoprocessorHost().preMergeRegions(regionsToMerge);
2299        String aid = getClientIdAuditPrefix();
2300        LOG.info("{} merge regions {}", aid, regionNamesToLog);
2301        submitProcedure(new MergeTableRegionsProcedure(procedureExecutor.getEnvironment(),
2302          regionsToMerge, forcible));
2303        getMaster().getMasterCoprocessorHost().postMergeRegions(regionsToMerge);
2304      }
2305
2306      @Override
2307      protected String getDescription() {
2308        return "MergeTableProcedure";
2309      }
2310    });
2311  }
2312
2313  @Override
2314  public long splitRegion(final RegionInfo regionInfo, final byte[] splitRow, final long nonceGroup,
2315    final long nonce) throws IOException {
2316    checkInitialized();
2317
2318    if (!isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
2319      LOG.warn("Split switch is off! skip split of " + regionInfo);
2320      throw new DoNotRetryIOException(
2321        "Split region " + regionInfo.getRegionNameAsString() + " failed due to split switch off");
2322    }
2323
2324    if (!getTableDescriptors().get(regionInfo.getTable()).isSplitEnabled()) {
2325      LOG.warn("Split is disabled for the table! Skipping split of {}", regionInfo);
2326      throw new DoNotRetryIOException("Split region " + regionInfo.getRegionNameAsString()
2327        + " failed as region split is disabled for the table");
2328    }
2329
2330    return MasterProcedureUtil
2331      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2332        @Override
2333        protected void run() throws IOException {
2334          getMaster().getMasterCoprocessorHost().preSplitRegion(regionInfo.getTable(), splitRow);
2335          LOG.info(getClientIdAuditPrefix() + " split " + regionInfo.getRegionNameAsString());
2336
2337          // Execute the operation asynchronously
2338          submitProcedure(getAssignmentManager().createSplitProcedure(regionInfo, splitRow));
2339        }
2340
2341        @Override
2342        protected String getDescription() {
2343          return "SplitTableProcedure";
2344        }
2345      });
2346  }
2347
2348  private void warmUpRegion(ServerName server, RegionInfo region) {
2349    FutureUtils.addListener(asyncClusterConnection.getRegionServerAdmin(server)
2350      .warmupRegion(RequestConverter.buildWarmupRegionRequest(region)), (r, e) -> {
2351        if (e != null) {
2352          LOG.warn("Failed to warm up region {} on server {}", region, server, e);
2353        }
2354      });
2355  }
2356
2357  // Public so can be accessed by tests. Blocks until move is done.
2358  // Replace with an async implementation from which you can get
2359  // a success/failure result.
2360  @InterfaceAudience.Private
2361  public void move(final byte[] encodedRegionName, byte[] destServerName) throws IOException {
2362    RegionState regionState =
2363      assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));
2364
2365    RegionInfo hri;
2366    if (regionState != null) {
2367      hri = regionState.getRegion();
2368    } else {
2369      throw new UnknownRegionException(Bytes.toStringBinary(encodedRegionName));
2370    }
2371
2372    ServerName dest;
2373    List<ServerName> exclude = hri.getTable().isSystemTable()
2374      ? assignmentManager.getExcludedServersForSystemTable()
2375      : new ArrayList<>(1);
2376    if (
2377      destServerName != null && exclude.contains(ServerName.valueOf(Bytes.toString(destServerName)))
2378    ) {
2379      LOG.info(Bytes.toString(encodedRegionName) + " can not move to "
2380        + Bytes.toString(destServerName) + " because the server is in exclude list");
2381      destServerName = null;
2382    }
2383    if (destServerName == null || destServerName.length == 0) {
2384      LOG.info("Passed destination servername is null/empty so " + "choosing a server at random");
2385      exclude.add(regionState.getServerName());
2386      final List<ServerName> destServers = this.serverManager.createDestinationServersList(exclude);
2387      dest = balancer.randomAssignment(hri, destServers);
2388      if (dest == null) {
2389        LOG.debug("Unable to determine a plan to assign " + hri);
2390        return;
2391      }
2392    } else {
2393      ServerName candidate = ServerName.valueOf(Bytes.toString(destServerName));
2394      dest = balancer.randomAssignment(hri, Lists.newArrayList(candidate));
2395      if (dest == null) {
2396        LOG.debug("Unable to determine a plan to assign " + hri);
2397        return;
2398      }
2399      // TODO: deal with table on master for rs group.
2400      if (dest.equals(serverName)) {
2401        // To avoid unnecessary region moving later by balancer. Don't put user
2402        // regions on master.
2403        LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
2404          + " to avoid unnecessary region moving later by load balancer,"
2405          + " because it should not be on master");
2406        return;
2407      }
2408    }
2409
2410    if (dest.equals(regionState.getServerName())) {
2411      LOG.debug("Skipping move of region " + hri.getRegionNameAsString()
2412        + " because region already assigned to the same server " + dest + ".");
2413      return;
2414    }
2415
2416    // Now we can do the move
2417    RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), dest);
2418    assert rp.getDestination() != null : rp.toString() + " " + dest;
2419
2420    try {
2421      checkInitialized();
2422      if (this.cpHost != null) {
2423        this.cpHost.preMove(hri, rp.getSource(), rp.getDestination());
2424      }
2425
2426      TransitRegionStateProcedure proc =
2427        this.assignmentManager.createMoveRegionProcedure(rp.getRegionInfo(), rp.getDestination());
2428      if (conf.getBoolean(WARMUP_BEFORE_MOVE, DEFAULT_WARMUP_BEFORE_MOVE)) {
2429        // Warmup the region on the destination before initiating the move.
2430        // A region server could reject the close request because it either does not
2431        // have the specified region or the region is being split.
2432        LOG.info(getClientIdAuditPrefix() + " move " + rp + ", warming up region on "
2433          + rp.getDestination());
2434        warmUpRegion(rp.getDestination(), hri);
2435      }
2436      LOG.info(getClientIdAuditPrefix() + " move " + rp + ", running balancer");
2437      Future<byte[]> future = ProcedureSyncWait.submitProcedure(this.procedureExecutor, proc);
2438      try {
2439        // Is this going to work? Will we throw exception on error?
2440        // TODO: CompletableFuture rather than this stunted Future.
2441        future.get();
2442      } catch (InterruptedException | ExecutionException e) {
2443        throw new HBaseIOException(e);
2444      }
2445      if (this.cpHost != null) {
2446        this.cpHost.postMove(hri, rp.getSource(), rp.getDestination());
2447      }
2448    } catch (IOException ioe) {
2449      if (ioe instanceof HBaseIOException) {
2450        throw (HBaseIOException) ioe;
2451      }
2452      throw new HBaseIOException(ioe);
2453    }
2454  }
2455
2456  @Override
2457  public long createTable(final TableDescriptor tableDescriptor, final byte[][] splitKeys,
2458    final long nonceGroup, final long nonce) throws IOException {
2459    checkInitialized();
2460    TableDescriptor desc = getMasterCoprocessorHost().preCreateTableRegionsInfos(tableDescriptor);
2461    if (desc == null) {
2462      throw new IOException("Creation for " + tableDescriptor + " is canceled by CP");
2463    }
2464    String namespace = desc.getTableName().getNamespaceAsString();
2465    this.clusterSchemaService.getNamespace(namespace);
2466
2467    RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(desc, splitKeys);
2468    TableDescriptorChecker.sanityCheck(conf, desc);
2469
2470    return MasterProcedureUtil
2471      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2472        @Override
2473        protected void run() throws IOException {
2474          getMaster().getMasterCoprocessorHost().preCreateTable(desc, newRegions);
2475
2476          LOG.info(getClientIdAuditPrefix() + " create " + desc);
2477
2478          // TODO: We can handle/merge duplicate requests, and differentiate the case of
2479          // TableExistsException by saying if the schema is the same or not.
2480          //
2481          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2482          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2483          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2484          submitProcedure(
2485            new CreateTableProcedure(procedureExecutor.getEnvironment(), desc, newRegions, latch));
2486          latch.await();
2487
2488          getMaster().getMasterCoprocessorHost().postCreateTable(desc, newRegions);
2489        }
2490
2491        @Override
2492        protected String getDescription() {
2493          return "CreateTableProcedure";
2494        }
2495      });
2496  }
2497
2498  @Override
2499  public long createSystemTable(final TableDescriptor tableDescriptor) throws IOException {
2500    if (isStopped()) {
2501      throw new MasterNotRunningException();
2502    }
2503
2504    TableName tableName = tableDescriptor.getTableName();
2505    if (!(tableName.isSystemTable())) {
2506      throw new IllegalArgumentException(
2507        "Only system table creation can use this createSystemTable API");
2508    }
2509
2510    RegionInfo[] newRegions = ModifyRegionUtils.createRegionInfos(tableDescriptor, null);
2511
2512    LOG.info(getClientIdAuditPrefix() + " create " + tableDescriptor);
2513
2514    // This special create table is called locally to master. Therefore, no RPC means no need
2515    // to use nonce to detect duplicated RPC call.
2516    long procId = this.procedureExecutor.submitProcedure(
2517      new CreateTableProcedure(procedureExecutor.getEnvironment(), tableDescriptor, newRegions));
2518
2519    return procId;
2520  }
2521
2522  private void startActiveMasterManager(int infoPort) throws KeeperException {
2523    String backupZNode = ZNodePaths.joinZNode(zooKeeper.getZNodePaths().backupMasterAddressesZNode,
2524      serverName.toString());
2525    /*
2526     * Add a ZNode for ourselves in the backup master directory since we may not become the active
2527     * master. If so, we want the actual active master to know we are backup masters, so that it
2528     * won't assign regions to us if so configured. If we become the active master later,
2529     * ActiveMasterManager will delete this node explicitly. If we crash before then, ZooKeeper will
2530     * delete this node for us since it is ephemeral.
2531     */
2532    LOG.info("Adding backup master ZNode " + backupZNode);
2533    if (!MasterAddressTracker.setMasterAddress(zooKeeper, backupZNode, serverName, infoPort)) {
2534      LOG.warn("Failed create of " + backupZNode + " by " + serverName);
2535    }
2536    this.activeMasterManager.setInfoPort(infoPort);
2537    int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
2538    // If we're a backup master, stall until a primary to write this address
2539    if (conf.getBoolean(HConstants.MASTER_TYPE_BACKUP, HConstants.DEFAULT_MASTER_TYPE_BACKUP)) {
2540      LOG.debug("HMaster started in backup mode. Stalling until master znode is written.");
2541      // This will only be a minute or so while the cluster starts up,
2542      // so don't worry about setting watches on the parent znode
2543      while (!activeMasterManager.hasActiveMaster()) {
2544        LOG.debug("Waiting for master address and cluster state znode to be written.");
2545        Threads.sleep(timeout);
2546      }
2547    }
2548
2549    // Here for the master startup process, we use TaskGroup to monitor the whole progress.
2550    // The UI is similar to how Hadoop designed the startup page for the NameNode.
2551    // See HBASE-21521 for more details.
2552    // We do not cleanup the startupTaskGroup, let the startup progress information
2553    // be permanent in the MEM.
2554    startupTaskGroup = TaskMonitor.createTaskGroup(true, "Master startup");
2555    try {
2556      if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
2557        finishActiveMasterInitialization();
2558      }
2559    } catch (Throwable t) {
2560      startupTaskGroup.abort("Failed to become active master due to:" + t.getMessage());
2561      LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
2562      // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
2563      if (
2564        t instanceof NoClassDefFoundError
2565          && t.getMessage().contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")
2566      ) {
2567        // improved error message for this special case
2568        abort("HBase is having a problem with its Hadoop jars.  You may need to recompile "
2569          + "HBase against Hadoop version " + org.apache.hadoop.util.VersionInfo.getVersion()
2570          + " or change your hadoop jars to start properly", t);
2571      } else {
2572        abort("Unhandled exception. Starting shutdown.", t);
2573      }
2574    }
2575  }
2576
2577  private static boolean isCatalogTable(final TableName tableName) {
2578    return tableName.equals(TableName.META_TABLE_NAME);
2579  }
2580
2581  @Override
2582  public long deleteTable(final TableName tableName, final long nonceGroup, final long nonce)
2583    throws IOException {
2584    checkInitialized();
2585
2586    return MasterProcedureUtil
2587      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2588        @Override
2589        protected void run() throws IOException {
2590          getMaster().getMasterCoprocessorHost().preDeleteTable(tableName);
2591
2592          LOG.info(getClientIdAuditPrefix() + " delete " + tableName);
2593
2594          // TODO: We can handle/merge duplicate request
2595          //
2596          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2597          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2598          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2599          submitProcedure(
2600            new DeleteTableProcedure(procedureExecutor.getEnvironment(), tableName, latch));
2601          latch.await();
2602
2603          getMaster().getMasterCoprocessorHost().postDeleteTable(tableName);
2604        }
2605
2606        @Override
2607        protected String getDescription() {
2608          return "DeleteTableProcedure";
2609        }
2610      });
2611  }
2612
2613  @Override
2614  public long truncateTable(final TableName tableName, final boolean preserveSplits,
2615    final long nonceGroup, final long nonce) throws IOException {
2616    checkInitialized();
2617
2618    return MasterProcedureUtil
2619      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2620        @Override
2621        protected void run() throws IOException {
2622          getMaster().getMasterCoprocessorHost().preTruncateTable(tableName);
2623
2624          LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
2625          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
2626          submitProcedure(new TruncateTableProcedure(procedureExecutor.getEnvironment(), tableName,
2627            preserveSplits, latch));
2628          latch.await();
2629
2630          getMaster().getMasterCoprocessorHost().postTruncateTable(tableName);
2631        }
2632
2633        @Override
2634        protected String getDescription() {
2635          return "TruncateTableProcedure";
2636        }
2637      });
2638  }
2639
2640  @Override
2641  public long truncateRegion(final RegionInfo regionInfo, final long nonceGroup, final long nonce)
2642    throws IOException {
2643    checkInitialized();
2644
2645    return MasterProcedureUtil
2646      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2647        @Override
2648        protected void run() throws IOException {
2649          getMaster().getMasterCoprocessorHost().preTruncateRegion(regionInfo);
2650
2651          LOG.info(
2652            getClientIdAuditPrefix() + " truncate region " + regionInfo.getRegionNameAsString());
2653
2654          // Execute the operation asynchronously
2655          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createLatch(2, 0);
2656          submitProcedure(
2657            new TruncateRegionProcedure(procedureExecutor.getEnvironment(), regionInfo, latch));
2658          latch.await();
2659
2660          getMaster().getMasterCoprocessorHost().postTruncateRegion(regionInfo);
2661        }
2662
2663        @Override
2664        protected String getDescription() {
2665          return "TruncateRegionProcedure";
2666        }
2667      });
2668  }
2669
2670  @Override
2671  public long addColumn(final TableName tableName, final ColumnFamilyDescriptor column,
2672    final long nonceGroup, final long nonce) throws IOException {
2673    checkInitialized();
2674    checkTableExists(tableName);
2675
2676    return modifyTable(tableName, new TableDescriptorGetter() {
2677
2678      @Override
2679      public TableDescriptor get() throws IOException {
2680        TableDescriptor old = getTableDescriptors().get(tableName);
2681        if (old.hasColumnFamily(column.getName())) {
2682          throw new InvalidFamilyOperationException("Column family '" + column.getNameAsString()
2683            + "' in table '" + tableName + "' already exists so cannot be added");
2684        }
2685
2686        return TableDescriptorBuilder.newBuilder(old).setColumnFamily(column).build();
2687      }
2688    }, nonceGroup, nonce, true);
2689  }
2690
2691  /**
2692   * Implement to return TableDescriptor after pre-checks
2693   */
2694  protected interface TableDescriptorGetter {
2695    TableDescriptor get() throws IOException;
2696  }
2697
2698  @Override
2699  public long modifyColumn(final TableName tableName, final ColumnFamilyDescriptor descriptor,
2700    final long nonceGroup, final long nonce) throws IOException {
2701    checkInitialized();
2702    checkTableExists(tableName);
2703    return modifyTable(tableName, new TableDescriptorGetter() {
2704
2705      @Override
2706      public TableDescriptor get() throws IOException {
2707        TableDescriptor old = getTableDescriptors().get(tableName);
2708        if (!old.hasColumnFamily(descriptor.getName())) {
2709          throw new InvalidFamilyOperationException("Family '" + descriptor.getNameAsString()
2710            + "' does not exist, so it cannot be modified");
2711        }
2712
2713        return TableDescriptorBuilder.newBuilder(old).modifyColumnFamily(descriptor).build();
2714      }
2715    }, nonceGroup, nonce, true);
2716  }
2717
2718  @Override
2719  public long modifyColumnStoreFileTracker(TableName tableName, byte[] family, String dstSFT,
2720    long nonceGroup, long nonce) throws IOException {
2721    checkInitialized();
2722    return MasterProcedureUtil
2723      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2724
2725        @Override
2726        protected void run() throws IOException {
2727          String sft = getMaster().getMasterCoprocessorHost()
2728            .preModifyColumnFamilyStoreFileTracker(tableName, family, dstSFT);
2729          LOG.info("{} modify column {} store file tracker of table {} to {}",
2730            getClientIdAuditPrefix(), Bytes.toStringBinary(family), tableName, sft);
2731          submitProcedure(new ModifyColumnFamilyStoreFileTrackerProcedure(
2732            procedureExecutor.getEnvironment(), tableName, family, sft));
2733          getMaster().getMasterCoprocessorHost().postModifyColumnFamilyStoreFileTracker(tableName,
2734            family, dstSFT);
2735        }
2736
2737        @Override
2738        protected String getDescription() {
2739          return "ModifyColumnFamilyStoreFileTrackerProcedure";
2740        }
2741      });
2742  }
2743
2744  @Override
2745  public long deleteColumn(final TableName tableName, final byte[] columnName,
2746    final long nonceGroup, final long nonce) throws IOException {
2747    checkInitialized();
2748    checkTableExists(tableName);
2749
2750    return modifyTable(tableName, new TableDescriptorGetter() {
2751
2752      @Override
2753      public TableDescriptor get() throws IOException {
2754        TableDescriptor old = getTableDescriptors().get(tableName);
2755
2756        if (!old.hasColumnFamily(columnName)) {
2757          throw new InvalidFamilyOperationException(
2758            "Family '" + Bytes.toString(columnName) + "' does not exist, so it cannot be deleted");
2759        }
2760        if (old.getColumnFamilyCount() == 1) {
2761          throw new InvalidFamilyOperationException("Family '" + Bytes.toString(columnName)
2762            + "' is the only column family in the table, so it cannot be deleted");
2763        }
2764        return TableDescriptorBuilder.newBuilder(old).removeColumnFamily(columnName).build();
2765      }
2766    }, nonceGroup, nonce, true);
2767  }
2768
2769  @Override
2770  public long enableTable(final TableName tableName, final long nonceGroup, final long nonce)
2771    throws IOException {
2772    checkInitialized();
2773
2774    return MasterProcedureUtil
2775      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2776        @Override
2777        protected void run() throws IOException {
2778          getMaster().getMasterCoprocessorHost().preEnableTable(tableName);
2779
2780          // Normally, it would make sense for this authorization check to exist inside
2781          // AccessController, but because the authorization check is done based on internal state
2782          // (rather than explicit permissions) we'll do the check here instead of in the
2783          // coprocessor.
2784          MasterQuotaManager quotaManager = getMasterQuotaManager();
2785          if (quotaManager != null) {
2786            if (quotaManager.isQuotaInitialized()) {
2787              // skip checking quotas for system tables, see:
2788              // https://issues.apache.org/jira/browse/HBASE-28183
2789              if (!tableName.isSystemTable()) {
2790                SpaceQuotaSnapshot currSnapshotOfTable =
2791                  QuotaTableUtil.getCurrentSnapshotFromQuotaTable(getConnection(), tableName);
2792                if (currSnapshotOfTable != null) {
2793                  SpaceQuotaStatus quotaStatus = currSnapshotOfTable.getQuotaStatus();
2794                  if (
2795                    quotaStatus.isInViolation()
2796                      && SpaceViolationPolicy.DISABLE == quotaStatus.getPolicy().orElse(null)
2797                  ) {
2798                    throw new AccessDeniedException("Enabling the table '" + tableName
2799                      + "' is disallowed due to a violated space quota.");
2800                  }
2801                }
2802              }
2803            } else if (LOG.isTraceEnabled()) {
2804              LOG
2805                .trace("Unable to check for space quotas as the MasterQuotaManager is not enabled");
2806            }
2807          }
2808
2809          LOG.info(getClientIdAuditPrefix() + " enable " + tableName);
2810
2811          // Execute the operation asynchronously - client will check the progress of the operation
2812          // In case the request is from a <1.1 client before returning,
2813          // we want to make sure that the table is prepared to be
2814          // enabled (the table is locked and the table state is set).
2815          // Note: if the procedure throws exception, we will catch it and rethrow.
2816          final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createLatch();
2817          submitProcedure(
2818            new EnableTableProcedure(procedureExecutor.getEnvironment(), tableName, prepareLatch));
2819          prepareLatch.await();
2820
2821          getMaster().getMasterCoprocessorHost().postEnableTable(tableName);
2822        }
2823
2824        @Override
2825        protected String getDescription() {
2826          return "EnableTableProcedure";
2827        }
2828      });
2829  }
2830
2831  @Override
2832  public long disableTable(final TableName tableName, final long nonceGroup, final long nonce)
2833    throws IOException {
2834    checkInitialized();
2835
2836    return MasterProcedureUtil
2837      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2838        @Override
2839        protected void run() throws IOException {
2840          getMaster().getMasterCoprocessorHost().preDisableTable(tableName);
2841
2842          LOG.info(getClientIdAuditPrefix() + " disable " + tableName);
2843
2844          // Execute the operation asynchronously - client will check the progress of the operation
2845          // In case the request is from a <1.1 client before returning,
2846          // we want to make sure that the table is prepared to be
2847          // enabled (the table is locked and the table state is set).
2848          // Note: if the procedure throws exception, we will catch it and rethrow.
2849          //
2850          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2851          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2852          final ProcedurePrepareLatch prepareLatch = ProcedurePrepareLatch.createBlockingLatch();
2853          submitProcedure(new DisableTableProcedure(procedureExecutor.getEnvironment(), tableName,
2854            false, prepareLatch));
2855          prepareLatch.await();
2856
2857          getMaster().getMasterCoprocessorHost().postDisableTable(tableName);
2858        }
2859
2860        @Override
2861        protected String getDescription() {
2862          return "DisableTableProcedure";
2863        }
2864      });
2865  }
2866
2867  private long modifyTable(final TableName tableName,
2868    final TableDescriptorGetter newDescriptorGetter, final long nonceGroup, final long nonce,
2869    final boolean shouldCheckDescriptor) throws IOException {
2870    return modifyTable(tableName, newDescriptorGetter, nonceGroup, nonce, shouldCheckDescriptor,
2871      true);
2872  }
2873
2874  private long modifyTable(final TableName tableName,
2875    final TableDescriptorGetter newDescriptorGetter, final long nonceGroup, final long nonce,
2876    final boolean shouldCheckDescriptor, final boolean reopenRegions) throws IOException {
2877    return MasterProcedureUtil
2878      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2879        @Override
2880        protected void run() throws IOException {
2881          TableDescriptor oldDescriptor = getMaster().getTableDescriptors().get(tableName);
2882          TableDescriptor newDescriptor = getMaster().getMasterCoprocessorHost()
2883            .preModifyTable(tableName, oldDescriptor, newDescriptorGetter.get());
2884          TableDescriptorChecker.sanityCheck(conf, newDescriptor);
2885          LOG.info("{} modify table {} from {} to {}", getClientIdAuditPrefix(), tableName,
2886            oldDescriptor, newDescriptor);
2887
2888          // Execute the operation synchronously - wait for the operation completes before
2889          // continuing.
2890          //
2891          // We need to wait for the procedure to potentially fail due to "prepare" sanity
2892          // checks. This will block only the beginning of the procedure. See HBASE-19953.
2893          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
2894          submitProcedure(new ModifyTableProcedure(procedureExecutor.getEnvironment(),
2895            newDescriptor, latch, oldDescriptor, shouldCheckDescriptor, reopenRegions));
2896          latch.await();
2897
2898          getMaster().getMasterCoprocessorHost().postModifyTable(tableName, oldDescriptor,
2899            newDescriptor);
2900        }
2901
2902        @Override
2903        protected String getDescription() {
2904          return "ModifyTableProcedure";
2905        }
2906      });
2907
2908  }
2909
2910  @Override
2911  public long modifyTable(final TableName tableName, final TableDescriptor newDescriptor,
2912    final long nonceGroup, final long nonce, final boolean reopenRegions) throws IOException {
2913    checkInitialized();
2914    return modifyTable(tableName, new TableDescriptorGetter() {
2915      @Override
2916      public TableDescriptor get() throws IOException {
2917        return newDescriptor;
2918      }
2919    }, nonceGroup, nonce, false, reopenRegions);
2920
2921  }
2922
2923  @Override
2924  public long modifyTableStoreFileTracker(TableName tableName, String dstSFT, long nonceGroup,
2925    long nonce) throws IOException {
2926    checkInitialized();
2927    return MasterProcedureUtil
2928      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2929
2930        @Override
2931        protected void run() throws IOException {
2932          String sft = getMaster().getMasterCoprocessorHost()
2933            .preModifyTableStoreFileTracker(tableName, dstSFT);
2934          LOG.info("{} modify table store file tracker of table {} to {}", getClientIdAuditPrefix(),
2935            tableName, sft);
2936          submitProcedure(new ModifyTableStoreFileTrackerProcedure(
2937            procedureExecutor.getEnvironment(), tableName, sft));
2938          getMaster().getMasterCoprocessorHost().postModifyTableStoreFileTracker(tableName, sft);
2939        }
2940
2941        @Override
2942        protected String getDescription() {
2943          return "ModifyTableStoreFileTrackerProcedure";
2944        }
2945      });
2946  }
2947
2948  public long restoreSnapshot(final SnapshotDescription snapshotDesc, final long nonceGroup,
2949    final long nonce, final boolean restoreAcl, final String customSFT) throws IOException {
2950    checkInitialized();
2951    getSnapshotManager().checkSnapshotSupport();
2952
2953    // Ensure namespace exists. Will throw exception if non-known NS.
2954    final TableName dstTable = TableName.valueOf(snapshotDesc.getTable());
2955    getClusterSchema().getNamespace(dstTable.getNamespaceAsString());
2956
2957    return MasterProcedureUtil
2958      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
2959        @Override
2960        protected void run() throws IOException {
2961          setProcId(getSnapshotManager().restoreOrCloneSnapshot(snapshotDesc, getNonceKey(),
2962            restoreAcl, customSFT));
2963        }
2964
2965        @Override
2966        protected String getDescription() {
2967          return "RestoreSnapshotProcedure";
2968        }
2969      });
2970  }
2971
2972  private void checkTableExists(final TableName tableName)
2973    throws IOException, TableNotFoundException {
2974    if (!tableDescriptors.exists(tableName)) {
2975      throw new TableNotFoundException(tableName);
2976    }
2977  }
2978
2979  @Override
2980  public void checkTableModifiable(final TableName tableName)
2981    throws IOException, TableNotFoundException, TableNotDisabledException {
2982    if (isCatalogTable(tableName)) {
2983      throw new IOException("Can't modify catalog tables");
2984    }
2985    checkTableExists(tableName);
2986    TableState ts = getTableStateManager().getTableState(tableName);
2987    if (!ts.isDisabled()) {
2988      throw new TableNotDisabledException("Not DISABLED; " + ts);
2989    }
2990  }
2991
2992  public ClusterMetrics getClusterMetricsWithoutCoprocessor() throws InterruptedIOException {
2993    return getClusterMetricsWithoutCoprocessor(EnumSet.allOf(Option.class));
2994  }
2995
2996  public ClusterMetrics getClusterMetricsWithoutCoprocessor(EnumSet<Option> options)
2997    throws InterruptedIOException {
2998    ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
2999    // given that hbase1 can't submit the request with Option,
3000    // we return all information to client if the list of Option is empty.
3001    if (options.isEmpty()) {
3002      options = EnumSet.allOf(Option.class);
3003    }
3004
3005    // TASKS and/or LIVE_SERVERS will populate this map, which will be given to the builder if
3006    // not null after option processing completes.
3007    Map<ServerName, ServerMetrics> serverMetricsMap = null;
3008
3009    for (Option opt : options) {
3010      switch (opt) {
3011        case HBASE_VERSION:
3012          builder.setHBaseVersion(VersionInfo.getVersion());
3013          break;
3014        case CLUSTER_ID:
3015          builder.setClusterId(getClusterId());
3016          break;
3017        case MASTER:
3018          builder.setMasterName(getServerName());
3019          break;
3020        case BACKUP_MASTERS:
3021          builder.setBackerMasterNames(getBackupMasters());
3022          break;
3023        case TASKS: {
3024          // Master tasks
3025          builder.setMasterTasks(TaskMonitor.get().getTasks().stream()
3026            .map(task -> ServerTaskBuilder.newBuilder().setDescription(task.getDescription())
3027              .setStatus(task.getStatus())
3028              .setState(ServerTask.State.valueOf(task.getState().name()))
3029              .setStartTime(task.getStartTime()).setCompletionTime(task.getCompletionTimestamp())
3030              .build())
3031            .collect(Collectors.toList()));
3032          // TASKS is also synonymous with LIVE_SERVERS for now because task information for
3033          // regionservers is carried in ServerLoad.
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 LIVE_SERVERS: {
3041          // Add entries to serverMetricsMap for all live servers, if we haven't already done so
3042          if (serverMetricsMap == null) {
3043            serverMetricsMap = getOnlineServers();
3044          }
3045          break;
3046        }
3047        case DEAD_SERVERS: {
3048          if (serverManager != null) {
3049            builder.setDeadServerNames(
3050              new ArrayList<>(serverManager.getDeadServers().copyServerNames()));
3051          }
3052          break;
3053        }
3054        case UNKNOWN_SERVERS: {
3055          if (serverManager != null) {
3056            builder.setUnknownServerNames(getUnknownServers());
3057          }
3058          break;
3059        }
3060        case MASTER_COPROCESSORS: {
3061          if (cpHost != null) {
3062            builder.setMasterCoprocessorNames(Arrays.asList(getMasterCoprocessors()));
3063          }
3064          break;
3065        }
3066        case REGIONS_IN_TRANSITION: {
3067          if (assignmentManager != null) {
3068            builder.setRegionsInTransition(
3069              assignmentManager.getRegionStates().getRegionsStateInTransition());
3070          }
3071          break;
3072        }
3073        case BALANCER_ON: {
3074          if (loadBalancerStateStore != null) {
3075            builder.setBalancerOn(loadBalancerStateStore.get());
3076          }
3077          break;
3078        }
3079        case MASTER_INFO_PORT: {
3080          if (infoServer != null) {
3081            builder.setMasterInfoPort(infoServer.getPort());
3082          }
3083          break;
3084        }
3085        case SERVERS_NAME: {
3086          if (serverManager != null) {
3087            builder.setServerNames(serverManager.getOnlineServersList());
3088          }
3089          break;
3090        }
3091        case TABLE_TO_REGIONS_COUNT: {
3092          if (isActiveMaster() && isInitialized() && assignmentManager != null) {
3093            try {
3094              Map<TableName, RegionStatesCount> tableRegionStatesCountMap = new HashMap<>();
3095              Map<String, TableDescriptor> tableDescriptorMap = getTableDescriptors().getAll();
3096              for (TableDescriptor tableDescriptor : tableDescriptorMap.values()) {
3097                TableName tableName = tableDescriptor.getTableName();
3098                RegionStatesCount regionStatesCount =
3099                  assignmentManager.getRegionStatesCount(tableName);
3100                tableRegionStatesCountMap.put(tableName, regionStatesCount);
3101              }
3102              builder.setTableRegionStatesCount(tableRegionStatesCountMap);
3103            } catch (IOException e) {
3104              LOG.error("Error while populating TABLE_TO_REGIONS_COUNT for Cluster Metrics..", e);
3105            }
3106          }
3107          break;
3108        }
3109        case DECOMMISSIONED_SERVERS: {
3110          if (serverManager != null) {
3111            builder.setDecommissionedServerNames(serverManager.getDrainingServersList());
3112          }
3113          break;
3114        }
3115      }
3116    }
3117
3118    if (serverMetricsMap != null) {
3119      builder.setLiveServerMetrics(serverMetricsMap);
3120    }
3121
3122    return builder.build();
3123  }
3124
3125  private List<ServerName> getUnknownServers() {
3126    if (serverManager != null) {
3127      final Set<ServerName> serverNames = getAssignmentManager().getRegionStates().getRegionStates()
3128        .stream().map(RegionState::getServerName).collect(Collectors.toSet());
3129      final List<ServerName> unknownServerNames = serverNames.stream()
3130        .filter(sn -> sn != null && serverManager.isServerUnknown(sn)).collect(Collectors.toList());
3131      return unknownServerNames;
3132    }
3133    return null;
3134  }
3135
3136  private Map<ServerName, ServerMetrics> getOnlineServers() {
3137    if (serverManager != null) {
3138      final Map<ServerName, ServerMetrics> map = new HashMap<>();
3139      serverManager.getOnlineServers().entrySet().forEach(e -> map.put(e.getKey(), e.getValue()));
3140      return map;
3141    }
3142    return null;
3143  }
3144
3145  /** Returns cluster status */
3146  public ClusterMetrics getClusterMetrics() throws IOException {
3147    return getClusterMetrics(EnumSet.allOf(Option.class));
3148  }
3149
3150  public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
3151    if (cpHost != null) {
3152      cpHost.preGetClusterMetrics();
3153    }
3154    ClusterMetrics status = getClusterMetricsWithoutCoprocessor(options);
3155    if (cpHost != null) {
3156      cpHost.postGetClusterMetrics(status);
3157    }
3158    return status;
3159  }
3160
3161  /** Returns info port of active master or 0 if any exception occurs. */
3162  public int getActiveMasterInfoPort() {
3163    return activeMasterManager.getActiveMasterInfoPort();
3164  }
3165
3166  /**
3167   * @param sn is ServerName of the backup master
3168   * @return info port of backup master or 0 if any exception occurs.
3169   */
3170  public int getBackupMasterInfoPort(final ServerName sn) {
3171    return activeMasterManager.getBackupMasterInfoPort(sn);
3172  }
3173
3174  /**
3175   * The set of loaded coprocessors is stored in a static set. Since it's statically allocated, it
3176   * does not require that HMaster's cpHost be initialized prior to accessing it.
3177   * @return a String representation of the set of names of the loaded coprocessors.
3178   */
3179  public static String getLoadedCoprocessors() {
3180    return CoprocessorHost.getLoadedCoprocessors().toString();
3181  }
3182
3183  /** Returns timestamp in millis when HMaster was started. */
3184  public long getMasterStartTime() {
3185    return startcode;
3186  }
3187
3188  /** Returns timestamp in millis when HMaster became the active master. */
3189  @Override
3190  public long getMasterActiveTime() {
3191    return masterActiveTime;
3192  }
3193
3194  /** Returns timestamp in millis when HMaster finished becoming the active master */
3195  public long getMasterFinishedInitializationTime() {
3196    return masterFinishedInitializationTime;
3197  }
3198
3199  public int getNumWALFiles() {
3200    return 0;
3201  }
3202
3203  public ProcedureStore getProcedureStore() {
3204    return procedureStore;
3205  }
3206
3207  public int getRegionServerInfoPort(final ServerName sn) {
3208    int port = this.serverManager.getInfoPort(sn);
3209    return port == 0
3210      ? conf.getInt(HConstants.REGIONSERVER_INFO_PORT, HConstants.DEFAULT_REGIONSERVER_INFOPORT)
3211      : port;
3212  }
3213
3214  @Override
3215  public String getRegionServerVersion(ServerName sn) {
3216    // Will return "0.0.0" if the server is not online to prevent move system region to unknown
3217    // version RS.
3218    return this.serverManager.getVersion(sn);
3219  }
3220
3221  @Override
3222  public void checkIfShouldMoveSystemRegionAsync() {
3223    assignmentManager.checkIfShouldMoveSystemRegionAsync();
3224  }
3225
3226  /** Returns array of coprocessor SimpleNames. */
3227  public String[] getMasterCoprocessors() {
3228    Set<String> masterCoprocessors = getMasterCoprocessorHost().getCoprocessors();
3229    return masterCoprocessors.toArray(new String[masterCoprocessors.size()]);
3230  }
3231
3232  @Override
3233  public void abort(String reason, Throwable cause) {
3234    if (!setAbortRequested() || isStopped()) {
3235      LOG.debug("Abort called but aborted={}, stopped={}", isAborted(), isStopped());
3236      return;
3237    }
3238    if (cpHost != null) {
3239      // HBASE-4014: dump a list of loaded coprocessors.
3240      LOG.error(HBaseMarkers.FATAL,
3241        "Master server abort: loaded coprocessors are: " + getLoadedCoprocessors());
3242    }
3243    String msg = "***** ABORTING master " + this + ": " + reason + " *****";
3244    if (cause != null) {
3245      LOG.error(HBaseMarkers.FATAL, msg, cause);
3246    } else {
3247      LOG.error(HBaseMarkers.FATAL, msg);
3248    }
3249
3250    try {
3251      stopMaster();
3252    } catch (IOException e) {
3253      LOG.error("Exception occurred while stopping master", e);
3254    }
3255  }
3256
3257  @Override
3258  public MasterCoprocessorHost getMasterCoprocessorHost() {
3259    return cpHost;
3260  }
3261
3262  @Override
3263  public MasterQuotaManager getMasterQuotaManager() {
3264    return quotaManager;
3265  }
3266
3267  @Override
3268  public ProcedureExecutor<MasterProcedureEnv> getMasterProcedureExecutor() {
3269    return procedureExecutor;
3270  }
3271
3272  @Override
3273  public ServerName getServerName() {
3274    return this.serverName;
3275  }
3276
3277  @Override
3278  public AssignmentManager getAssignmentManager() {
3279    return this.assignmentManager;
3280  }
3281
3282  @Override
3283  public CatalogJanitor getCatalogJanitor() {
3284    return this.catalogJanitorChore;
3285  }
3286
3287  public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
3288    return rsFatals;
3289  }
3290
3291  public TaskGroup getStartupProgress() {
3292    return startupTaskGroup;
3293  }
3294
3295  /**
3296   * Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
3297   */
3298  public void shutdown() throws IOException {
3299    TraceUtil.trace(() -> {
3300      if (cpHost != null) {
3301        cpHost.preShutdown();
3302      }
3303
3304      // Tell the servermanager cluster shutdown has been called. This makes it so when Master is
3305      // last running server, it'll stop itself. Next, we broadcast the cluster shutdown by setting
3306      // the cluster status as down. RegionServers will notice this change in state and will start
3307      // shutting themselves down. When last has exited, Master can go down.
3308      if (this.serverManager != null) {
3309        this.serverManager.shutdownCluster();
3310      }
3311      if (this.clusterStatusTracker != null) {
3312        try {
3313          this.clusterStatusTracker.setClusterDown();
3314        } catch (KeeperException e) {
3315          LOG.error("ZooKeeper exception trying to set cluster as down in ZK", e);
3316        }
3317      }
3318      // Stop the procedure executor. Will stop any ongoing assign, unassign, server crash etc.,
3319      // processing so we can go down.
3320      if (this.procedureExecutor != null) {
3321        this.procedureExecutor.stop();
3322      }
3323      // Shutdown our cluster connection. This will kill any hosted RPCs that might be going on;
3324      // this is what we want especially if the Master is in startup phase doing call outs to
3325      // hbase:meta, etc. when cluster is down. Without ths connection close, we'd have to wait on
3326      // the rpc to timeout.
3327      if (this.asyncClusterConnection != null) {
3328        this.asyncClusterConnection.close();
3329      }
3330    }, "HMaster.shutdown");
3331  }
3332
3333  public void stopMaster() throws IOException {
3334    if (cpHost != null) {
3335      cpHost.preStopMaster();
3336    }
3337    stop("Stopped by " + Thread.currentThread().getName());
3338  }
3339
3340  @Override
3341  public void stop(String msg) {
3342    if (!this.stopped) {
3343      LOG.info("***** STOPPING master '" + this + "' *****");
3344      this.stopped = true;
3345      LOG.info("STOPPED: " + msg);
3346      // Wakes run() if it is sleeping
3347      sleeper.skipSleepCycle();
3348      if (this.activeMasterManager != null) {
3349        this.activeMasterManager.stop();
3350      }
3351    }
3352  }
3353
3354  protected void checkServiceStarted() throws ServerNotRunningYetException {
3355    if (!serviceStarted) {
3356      throw new ServerNotRunningYetException("Server is not running yet");
3357    }
3358  }
3359
3360  void checkInitialized() throws PleaseHoldException, ServerNotRunningYetException,
3361    MasterNotRunningException, MasterStoppedException {
3362    checkServiceStarted();
3363    if (!isInitialized()) {
3364      throw new PleaseHoldException("Master is initializing");
3365    }
3366    if (isStopped()) {
3367      throw new MasterStoppedException();
3368    }
3369  }
3370
3371  /**
3372   * Report whether this master is currently the active master or not. If not active master, we are
3373   * parked on ZK waiting to become active. This method is used for testing.
3374   * @return true if active master, false if not.
3375   */
3376  @Override
3377  public boolean isActiveMaster() {
3378    return activeMaster;
3379  }
3380
3381  /**
3382   * Report whether this master has completed with its initialization and is ready. If ready, the
3383   * master is also the active master. A standby master is never ready. This method is used for
3384   * testing.
3385   * @return true if master is ready to go, false if not.
3386   */
3387  @Override
3388  public boolean isInitialized() {
3389    return initialized.isReady();
3390  }
3391
3392  /**
3393   * Report whether this master is started This method is used for testing.
3394   * @return true if master is ready to go, false if not.
3395   */
3396  public boolean isOnline() {
3397    return serviceStarted;
3398  }
3399
3400  /**
3401   * Report whether this master is in maintenance mode.
3402   * @return true if master is in maintenanceMode
3403   */
3404  @Override
3405  public boolean isInMaintenanceMode() {
3406    return maintenanceMode;
3407  }
3408
3409  public void setInitialized(boolean isInitialized) {
3410    procedureExecutor.getEnvironment().setEventReady(initialized, isInitialized);
3411  }
3412
3413  /**
3414   * Mainly used in procedure related tests, where we will restart ProcedureExecutor and
3415   * AssignmentManager, but we do not want to restart master(to speed up the test), so we need to
3416   * disable rpc for a while otherwise some critical rpc requests such as
3417   * reportRegionStateTransition could fail and cause region server to abort.
3418   */
3419  @RestrictedApi(explanation = "Should only be called in tests", link = "",
3420      allowedOnPath = ".*/src/test/.*")
3421  public void setServiceStarted(boolean started) {
3422    this.serviceStarted = started;
3423  }
3424
3425  @Override
3426  public ProcedureEvent<?> getInitializedEvent() {
3427    return initialized;
3428  }
3429
3430  /**
3431   * Compute the average load across all region servers. Currently, this uses a very naive
3432   * computation - just uses the number of regions being served, ignoring stats about number of
3433   * requests.
3434   * @return the average load
3435   */
3436  public double getAverageLoad() {
3437    if (this.assignmentManager == null) {
3438      return 0;
3439    }
3440
3441    RegionStates regionStates = this.assignmentManager.getRegionStates();
3442    if (regionStates == null) {
3443      return 0;
3444    }
3445    return regionStates.getAverageLoad();
3446  }
3447
3448  @Override
3449  public boolean registerService(Service instance) {
3450    /*
3451     * No stacking of instances is allowed for a single service name
3452     */
3453    Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
3454    String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
3455    if (coprocessorServiceHandlers.containsKey(serviceName)) {
3456      LOG.error("Coprocessor service " + serviceName
3457        + " already registered, rejecting request from " + instance);
3458      return false;
3459    }
3460
3461    coprocessorServiceHandlers.put(serviceName, instance);
3462    if (LOG.isDebugEnabled()) {
3463      LOG.debug("Registered master coprocessor service: service=" + serviceName);
3464    }
3465    return true;
3466  }
3467
3468  /**
3469   * Utility for constructing an instance of the passed HMaster class.
3470   * @return HMaster instance.
3471   */
3472  public static HMaster constructMaster(Class<? extends HMaster> masterClass,
3473    final Configuration conf) {
3474    try {
3475      Constructor<? extends HMaster> c = masterClass.getConstructor(Configuration.class);
3476      return c.newInstance(conf);
3477    } catch (Exception e) {
3478      Throwable error = e;
3479      if (
3480        e instanceof InvocationTargetException
3481          && ((InvocationTargetException) e).getTargetException() != null
3482      ) {
3483        error = ((InvocationTargetException) e).getTargetException();
3484      }
3485      throw new RuntimeException("Failed construction of Master: " + masterClass.toString() + ". ",
3486        error);
3487    }
3488  }
3489
3490  /**
3491   * @see org.apache.hadoop.hbase.master.HMasterCommandLine
3492   */
3493  public static void main(String[] args) {
3494    LOG.info("STARTING service " + HMaster.class.getSimpleName());
3495    VersionInfo.logVersion();
3496    new HMasterCommandLine(HMaster.class).doMain(args);
3497  }
3498
3499  public HFileCleaner getHFileCleaner() {
3500    return this.hfileCleaners.get(0);
3501  }
3502
3503  public List<HFileCleaner> getHFileCleaners() {
3504    return this.hfileCleaners;
3505  }
3506
3507  public LogCleaner getLogCleaner() {
3508    return this.logCleaner;
3509  }
3510
3511  /** Returns the underlying snapshot manager */
3512  @Override
3513  public SnapshotManager getSnapshotManager() {
3514    return this.snapshotManager;
3515  }
3516
3517  /** Returns the underlying MasterProcedureManagerHost */
3518  @Override
3519  public MasterProcedureManagerHost getMasterProcedureManagerHost() {
3520    return mpmHost;
3521  }
3522
3523  @Override
3524  public ClusterSchema getClusterSchema() {
3525    return this.clusterSchemaService;
3526  }
3527
3528  /**
3529   * Create a new Namespace.
3530   * @param namespaceDescriptor descriptor for new Namespace
3531   * @param nonceGroup          Identifier for the source of the request, a client or process.
3532   * @param nonce               A unique identifier for this operation from the client or process
3533   *                            identified by <code>nonceGroup</code> (the source must ensure each
3534   *                            operation gets a unique id).
3535   * @return procedure id
3536   */
3537  long createNamespace(final NamespaceDescriptor namespaceDescriptor, final long nonceGroup,
3538    final long nonce) throws IOException {
3539    checkInitialized();
3540
3541    TableName.isLegalNamespaceName(Bytes.toBytes(namespaceDescriptor.getName()));
3542
3543    return MasterProcedureUtil
3544      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
3545        @Override
3546        protected void run() throws IOException {
3547          getMaster().getMasterCoprocessorHost().preCreateNamespace(namespaceDescriptor);
3548          // We need to wait for the procedure to potentially fail due to "prepare" sanity
3549          // checks. This will block only the beginning of the procedure. See HBASE-19953.
3550          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3551          LOG.info(getClientIdAuditPrefix() + " creating " + namespaceDescriptor);
3552          // Execute the operation synchronously - wait for the operation to complete before
3553          // continuing.
3554          setProcId(getClusterSchema().createNamespace(namespaceDescriptor, getNonceKey(), latch));
3555          latch.await();
3556          getMaster().getMasterCoprocessorHost().postCreateNamespace(namespaceDescriptor);
3557        }
3558
3559        @Override
3560        protected String getDescription() {
3561          return "CreateNamespaceProcedure";
3562        }
3563      });
3564  }
3565
3566  /**
3567   * Modify an existing Namespace.
3568   * @param nonceGroup Identifier for the source of the request, a client or process.
3569   * @param nonce      A unique identifier for this operation from the client or process identified
3570   *                   by <code>nonceGroup</code> (the source must ensure each operation gets a
3571   *                   unique id).
3572   * @return procedure id
3573   */
3574  long modifyNamespace(final NamespaceDescriptor newNsDescriptor, final long nonceGroup,
3575    final long nonce) throws IOException {
3576    checkInitialized();
3577
3578    TableName.isLegalNamespaceName(Bytes.toBytes(newNsDescriptor.getName()));
3579
3580    return MasterProcedureUtil
3581      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
3582        @Override
3583        protected void run() throws IOException {
3584          NamespaceDescriptor oldNsDescriptor = getNamespace(newNsDescriptor.getName());
3585          getMaster().getMasterCoprocessorHost().preModifyNamespace(oldNsDescriptor,
3586            newNsDescriptor);
3587          // We need to wait for the procedure to potentially fail due to "prepare" sanity
3588          // checks. This will block only the beginning of the procedure. See HBASE-19953.
3589          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3590          LOG.info(getClientIdAuditPrefix() + " modify " + newNsDescriptor);
3591          // Execute the operation synchronously - wait for the operation to complete before
3592          // continuing.
3593          setProcId(getClusterSchema().modifyNamespace(newNsDescriptor, getNonceKey(), latch));
3594          latch.await();
3595          getMaster().getMasterCoprocessorHost().postModifyNamespace(oldNsDescriptor,
3596            newNsDescriptor);
3597        }
3598
3599        @Override
3600        protected String getDescription() {
3601          return "ModifyNamespaceProcedure";
3602        }
3603      });
3604  }
3605
3606  /**
3607   * Delete an existing Namespace. Only empty Namespaces (no tables) can be removed.
3608   * @param nonceGroup Identifier for the source of the request, a client or process.
3609   * @param nonce      A unique identifier for this operation from the client or process identified
3610   *                   by <code>nonceGroup</code> (the source must ensure each operation gets a
3611   *                   unique id).
3612   * @return procedure id
3613   */
3614  long deleteNamespace(final String name, final long nonceGroup, final long nonce)
3615    throws IOException {
3616    checkInitialized();
3617
3618    return MasterProcedureUtil
3619      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
3620        @Override
3621        protected void run() throws IOException {
3622          getMaster().getMasterCoprocessorHost().preDeleteNamespace(name);
3623          LOG.info(getClientIdAuditPrefix() + " delete " + name);
3624          // Execute the operation synchronously - wait for the operation to complete before
3625          // continuing.
3626          //
3627          // We need to wait for the procedure to potentially fail due to "prepare" sanity
3628          // checks. This will block only the beginning of the procedure. See HBASE-19953.
3629          ProcedurePrepareLatch latch = ProcedurePrepareLatch.createBlockingLatch();
3630          setProcId(submitProcedure(
3631            new DeleteNamespaceProcedure(procedureExecutor.getEnvironment(), name, latch)));
3632          latch.await();
3633          // Will not be invoked in the face of Exception thrown by the Procedure's execution
3634          getMaster().getMasterCoprocessorHost().postDeleteNamespace(name);
3635        }
3636
3637        @Override
3638        protected String getDescription() {
3639          return "DeleteNamespaceProcedure";
3640        }
3641      });
3642  }
3643
3644  /**
3645   * Get a Namespace
3646   * @param name Name of the Namespace
3647   * @return Namespace descriptor for <code>name</code>
3648   */
3649  NamespaceDescriptor getNamespace(String name) throws IOException {
3650    checkInitialized();
3651    if (this.cpHost != null) this.cpHost.preGetNamespaceDescriptor(name);
3652    NamespaceDescriptor nsd = this.clusterSchemaService.getNamespace(name);
3653    if (this.cpHost != null) this.cpHost.postGetNamespaceDescriptor(nsd);
3654    return nsd;
3655  }
3656
3657  /**
3658   * Get all Namespaces
3659   * @return All Namespace descriptors
3660   */
3661  List<NamespaceDescriptor> getNamespaces() throws IOException {
3662    checkInitialized();
3663    final List<NamespaceDescriptor> nsds = new ArrayList<>();
3664    if (cpHost != null) {
3665      cpHost.preListNamespaceDescriptors(nsds);
3666    }
3667    nsds.addAll(this.clusterSchemaService.getNamespaces());
3668    if (this.cpHost != null) {
3669      this.cpHost.postListNamespaceDescriptors(nsds);
3670    }
3671    return nsds;
3672  }
3673
3674  /**
3675   * List namespace names
3676   * @return All namespace names
3677   */
3678  public List<String> listNamespaces() throws IOException {
3679    checkInitialized();
3680    List<String> namespaces = new ArrayList<>();
3681    if (cpHost != null) {
3682      cpHost.preListNamespaces(namespaces);
3683    }
3684    for (NamespaceDescriptor namespace : clusterSchemaService.getNamespaces()) {
3685      namespaces.add(namespace.getName());
3686    }
3687    if (cpHost != null) {
3688      cpHost.postListNamespaces(namespaces);
3689    }
3690    return namespaces;
3691  }
3692
3693  @Override
3694  public List<TableName> listTableNamesByNamespace(String name) throws IOException {
3695    checkInitialized();
3696    return listTableNames(name, null, true);
3697  }
3698
3699  @Override
3700  public List<TableDescriptor> listTableDescriptorsByNamespace(String name) throws IOException {
3701    checkInitialized();
3702    return listTableDescriptors(name, null, null, true);
3703  }
3704
3705  @Override
3706  public boolean abortProcedure(final long procId, final boolean mayInterruptIfRunning)
3707    throws IOException {
3708    if (cpHost != null) {
3709      cpHost.preAbortProcedure(this.procedureExecutor, procId);
3710    }
3711
3712    final boolean result = this.procedureExecutor.abort(procId, mayInterruptIfRunning);
3713
3714    if (cpHost != null) {
3715      cpHost.postAbortProcedure();
3716    }
3717
3718    return result;
3719  }
3720
3721  @Override
3722  public List<Procedure<?>> getProcedures() throws IOException {
3723    if (cpHost != null) {
3724      cpHost.preGetProcedures();
3725    }
3726
3727    @SuppressWarnings({ "unchecked", "rawtypes" })
3728    List<Procedure<?>> procList = (List) this.procedureExecutor.getProcedures();
3729
3730    if (cpHost != null) {
3731      cpHost.postGetProcedures(procList);
3732    }
3733
3734    return procList;
3735  }
3736
3737  @Override
3738  public List<LockedResource> getLocks() throws IOException {
3739    if (cpHost != null) {
3740      cpHost.preGetLocks();
3741    }
3742
3743    MasterProcedureScheduler procedureScheduler =
3744      procedureExecutor.getEnvironment().getProcedureScheduler();
3745
3746    final List<LockedResource> lockedResources = procedureScheduler.getLocks();
3747
3748    if (cpHost != null) {
3749      cpHost.postGetLocks(lockedResources);
3750    }
3751
3752    return lockedResources;
3753  }
3754
3755  /**
3756   * Returns the list of table descriptors that match the specified request
3757   * @param namespace        the namespace to query, or null if querying for all
3758   * @param regex            The regular expression to match against, or null if querying for all
3759   * @param tableNameList    the list of table names, or null if querying for all
3760   * @param includeSysTables False to match only against userspace tables
3761   * @return the list of table descriptors
3762   */
3763  public List<TableDescriptor> listTableDescriptors(final String namespace, final String regex,
3764    final List<TableName> tableNameList, final boolean includeSysTables) throws IOException {
3765    List<TableDescriptor> htds = new ArrayList<>();
3766    if (cpHost != null) {
3767      cpHost.preGetTableDescriptors(tableNameList, htds, regex);
3768    }
3769    htds = getTableDescriptors(htds, namespace, regex, tableNameList, includeSysTables);
3770    if (cpHost != null) {
3771      cpHost.postGetTableDescriptors(tableNameList, htds, regex);
3772    }
3773    return htds;
3774  }
3775
3776  /**
3777   * Returns the list of table names that match the specified request
3778   * @param regex            The regular expression to match against, or null if querying for all
3779   * @param namespace        the namespace to query, or null if querying for all
3780   * @param includeSysTables False to match only against userspace tables
3781   * @return the list of table names
3782   */
3783  public List<TableName> listTableNames(final String namespace, final String regex,
3784    final boolean includeSysTables) throws IOException {
3785    List<TableDescriptor> htds = new ArrayList<>();
3786    if (cpHost != null) {
3787      cpHost.preGetTableNames(htds, regex);
3788    }
3789    htds = getTableDescriptors(htds, namespace, regex, null, includeSysTables);
3790    if (cpHost != null) {
3791      cpHost.postGetTableNames(htds, regex);
3792    }
3793    List<TableName> result = new ArrayList<>(htds.size());
3794    for (TableDescriptor htd : htds)
3795      result.add(htd.getTableName());
3796    return result;
3797  }
3798
3799  /**
3800   * Return a list of table table descriptors after applying any provided filter parameters. Note
3801   * that the user-facing description of this filter logic is presented on the class-level javadoc
3802   * of {@link NormalizeTableFilterParams}.
3803   */
3804  private List<TableDescriptor> getTableDescriptors(final List<TableDescriptor> htds,
3805    final String namespace, final String regex, final List<TableName> tableNameList,
3806    final boolean includeSysTables) throws IOException {
3807    if (tableNameList == null || tableNameList.isEmpty()) {
3808      // request for all TableDescriptors
3809      Collection<TableDescriptor> allHtds;
3810      if (namespace != null && namespace.length() > 0) {
3811        // Do a check on the namespace existence. Will fail if does not exist.
3812        this.clusterSchemaService.getNamespace(namespace);
3813        allHtds = tableDescriptors.getByNamespace(namespace).values();
3814      } else {
3815        allHtds = tableDescriptors.getAll().values();
3816      }
3817      for (TableDescriptor desc : allHtds) {
3818        if (
3819          tableStateManager.isTablePresent(desc.getTableName())
3820            && (includeSysTables || !desc.getTableName().isSystemTable())
3821        ) {
3822          htds.add(desc);
3823        }
3824      }
3825    } else {
3826      for (TableName s : tableNameList) {
3827        if (tableStateManager.isTablePresent(s)) {
3828          TableDescriptor desc = tableDescriptors.get(s);
3829          if (desc != null) {
3830            htds.add(desc);
3831          }
3832        }
3833      }
3834    }
3835
3836    // Retains only those matched by regular expression.
3837    if (regex != null) filterTablesByRegex(htds, Pattern.compile(regex));
3838    return htds;
3839  }
3840
3841  /**
3842   * Removes the table descriptors that don't match the pattern.
3843   * @param descriptors list of table descriptors to filter
3844   * @param pattern     the regex to use
3845   */
3846  private static void filterTablesByRegex(final Collection<TableDescriptor> descriptors,
3847    final Pattern pattern) {
3848    final String defaultNS = NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR;
3849    Iterator<TableDescriptor> itr = descriptors.iterator();
3850    while (itr.hasNext()) {
3851      TableDescriptor htd = itr.next();
3852      String tableName = htd.getTableName().getNameAsString();
3853      boolean matched = pattern.matcher(tableName).matches();
3854      if (!matched && htd.getTableName().getNamespaceAsString().equals(defaultNS)) {
3855        matched = pattern.matcher(defaultNS + TableName.NAMESPACE_DELIM + tableName).matches();
3856      }
3857      if (!matched) {
3858        itr.remove();
3859      }
3860    }
3861  }
3862
3863  @Override
3864  public long getLastMajorCompactionTimestamp(TableName table) throws IOException {
3865    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
3866      .getLastMajorCompactionTimestamp(table);
3867  }
3868
3869  @Override
3870  public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
3871    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
3872      .getLastMajorCompactionTimestamp(regionName);
3873  }
3874
3875  /**
3876   * Gets the mob file compaction state for a specific table. Whether all the mob files are selected
3877   * is known during the compaction execution, but the statistic is done just before compaction
3878   * starts, it is hard to know the compaction type at that time, so the rough statistics are chosen
3879   * for the mob file compaction. Only two compaction states are available,
3880   * CompactionState.MAJOR_AND_MINOR and CompactionState.NONE.
3881   * @param tableName The current table name.
3882   * @return If a given table is in mob file compaction now.
3883   */
3884  public GetRegionInfoResponse.CompactionState getMobCompactionState(TableName tableName) {
3885    AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3886    if (compactionsCount != null && compactionsCount.get() != 0) {
3887      return GetRegionInfoResponse.CompactionState.MAJOR_AND_MINOR;
3888    }
3889    return GetRegionInfoResponse.CompactionState.NONE;
3890  }
3891
3892  public void reportMobCompactionStart(TableName tableName) throws IOException {
3893    IdLock.Entry lockEntry = null;
3894    try {
3895      lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
3896      AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3897      if (compactionsCount == null) {
3898        compactionsCount = new AtomicInteger(0);
3899        mobCompactionStates.put(tableName, compactionsCount);
3900      }
3901      compactionsCount.incrementAndGet();
3902    } finally {
3903      if (lockEntry != null) {
3904        mobCompactionLock.releaseLockEntry(lockEntry);
3905      }
3906    }
3907  }
3908
3909  public void reportMobCompactionEnd(TableName tableName) throws IOException {
3910    IdLock.Entry lockEntry = null;
3911    try {
3912      lockEntry = mobCompactionLock.getLockEntry(tableName.hashCode());
3913      AtomicInteger compactionsCount = mobCompactionStates.get(tableName);
3914      if (compactionsCount != null) {
3915        int count = compactionsCount.decrementAndGet();
3916        // remove the entry if the count is 0.
3917        if (count == 0) {
3918          mobCompactionStates.remove(tableName);
3919        }
3920      }
3921    } finally {
3922      if (lockEntry != null) {
3923        mobCompactionLock.releaseLockEntry(lockEntry);
3924      }
3925    }
3926  }
3927
3928  /**
3929   * Queries the state of the {@link LoadBalancerStateStore}. If the balancer is not initialized,
3930   * false is returned.
3931   * @return The state of the load balancer, or false if the load balancer isn't defined.
3932   */
3933  public boolean isBalancerOn() {
3934    return !isInMaintenanceMode() && loadBalancerStateStore != null && loadBalancerStateStore.get();
3935  }
3936
3937  /**
3938   * Queries the state of the {@link RegionNormalizerStateStore}. If it's not initialized, false is
3939   * returned.
3940   */
3941  public boolean isNormalizerOn() {
3942    return !isInMaintenanceMode() && getRegionNormalizerManager().isNormalizerOn();
3943  }
3944
3945  /**
3946   * Queries the state of the {@link SplitOrMergeStateStore}. If it is not initialized, false is
3947   * returned. If switchType is illegal, false will return.
3948   * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}
3949   * @return The state of the switch
3950   */
3951  @Override
3952  public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
3953    return !isInMaintenanceMode() && splitOrMergeStateStore != null
3954      && splitOrMergeStateStore.isSplitOrMergeEnabled(switchType);
3955  }
3956
3957  /**
3958   * Fetch the configured {@link LoadBalancer} class name. If none is set, a default is returned.
3959   * <p/>
3960   * Notice that, the base load balancer will always be {@link RSGroupBasedLoadBalancer} now, so
3961   * this method will return the balancer used inside each rs group.
3962   * @return The name of the {@link LoadBalancer} in use.
3963   */
3964  public String getLoadBalancerClassName() {
3965    return conf.get(HConstants.HBASE_MASTER_LOADBALANCER_CLASS,
3966      LoadBalancerFactory.getDefaultLoadBalancerClass().getName());
3967  }
3968
3969  public SplitOrMergeStateStore getSplitOrMergeStateStore() {
3970    return splitOrMergeStateStore;
3971  }
3972
3973  @Override
3974  public RSGroupBasedLoadBalancer getLoadBalancer() {
3975    return balancer;
3976  }
3977
3978  @Override
3979  public FavoredNodesManager getFavoredNodesManager() {
3980    return balancer.getFavoredNodesManager();
3981  }
3982
3983  private long executePeerProcedure(AbstractPeerProcedure<?> procedure) throws IOException {
3984    if (!isReplicationPeerModificationEnabled()) {
3985      throw new IOException("Replication peer modification disabled");
3986    }
3987    long procId = procedureExecutor.submitProcedure(procedure);
3988    procedure.getLatch().await();
3989    return procId;
3990  }
3991
3992  @Override
3993  public long addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled)
3994    throws ReplicationException, IOException {
3995    LOG.info(getClientIdAuditPrefix() + " creating replication peer, id=" + peerId + ", config="
3996      + peerConfig + ", state=" + (enabled ? "ENABLED" : "DISABLED"));
3997    return executePeerProcedure(new AddPeerProcedure(peerId, peerConfig, enabled));
3998  }
3999
4000  @Override
4001  public long removeReplicationPeer(String peerId) throws ReplicationException, IOException {
4002    LOG.info(getClientIdAuditPrefix() + " removing replication peer, id=" + peerId);
4003    return executePeerProcedure(new RemovePeerProcedure(peerId));
4004  }
4005
4006  @Override
4007  public long enableReplicationPeer(String peerId) throws ReplicationException, IOException {
4008    LOG.info(getClientIdAuditPrefix() + " enable replication peer, id=" + peerId);
4009    return executePeerProcedure(new EnablePeerProcedure(peerId));
4010  }
4011
4012  @Override
4013  public long disableReplicationPeer(String peerId) throws ReplicationException, IOException {
4014    LOG.info(getClientIdAuditPrefix() + " disable replication peer, id=" + peerId);
4015    return executePeerProcedure(new DisablePeerProcedure(peerId));
4016  }
4017
4018  @Override
4019  public ReplicationPeerConfig getReplicationPeerConfig(String peerId)
4020    throws ReplicationException, IOException {
4021    if (cpHost != null) {
4022      cpHost.preGetReplicationPeerConfig(peerId);
4023    }
4024    LOG.info(getClientIdAuditPrefix() + " get replication peer config, id=" + peerId);
4025    ReplicationPeerConfig peerConfig = this.replicationPeerManager.getPeerConfig(peerId)
4026      .orElseThrow(() -> new ReplicationPeerNotFoundException(peerId));
4027    if (cpHost != null) {
4028      cpHost.postGetReplicationPeerConfig(peerId);
4029    }
4030    return peerConfig;
4031  }
4032
4033  @Override
4034  public long updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig)
4035    throws ReplicationException, IOException {
4036    LOG.info(getClientIdAuditPrefix() + " update replication peer config, id=" + peerId
4037      + ", config=" + peerConfig);
4038    return executePeerProcedure(new UpdatePeerConfigProcedure(peerId, peerConfig));
4039  }
4040
4041  @Override
4042  public List<ReplicationPeerDescription> listReplicationPeers(String regex)
4043    throws ReplicationException, IOException {
4044    if (cpHost != null) {
4045      cpHost.preListReplicationPeers(regex);
4046    }
4047    LOG.debug("{} list replication peers, regex={}", getClientIdAuditPrefix(), regex);
4048    Pattern pattern = regex == null ? null : Pattern.compile(regex);
4049    List<ReplicationPeerDescription> peers = this.replicationPeerManager.listPeers(pattern);
4050    if (cpHost != null) {
4051      cpHost.postListReplicationPeers(regex);
4052    }
4053    return peers;
4054  }
4055
4056  @Override
4057  public long transitReplicationPeerSyncReplicationState(String peerId, SyncReplicationState state)
4058    throws ReplicationException, IOException {
4059    LOG.info(
4060      getClientIdAuditPrefix()
4061        + " transit current cluster state to {} in a synchronous replication peer id={}",
4062      state, peerId);
4063    return executePeerProcedure(new TransitPeerSyncReplicationStateProcedure(peerId, state));
4064  }
4065
4066  @Override
4067  public boolean replicationPeerModificationSwitch(boolean on) throws IOException {
4068    return replicationPeerModificationStateStore.set(on);
4069  }
4070
4071  @Override
4072  public boolean isReplicationPeerModificationEnabled() {
4073    return replicationPeerModificationStateStore.get();
4074  }
4075
4076  /**
4077   * Mark region server(s) as decommissioned (previously called 'draining') to prevent additional
4078   * regions from getting assigned to them. Also unload the regions on the servers asynchronously.0
4079   * @param servers Region servers to decommission.
4080   */
4081  public void decommissionRegionServers(final List<ServerName> servers, final boolean offload)
4082    throws IOException {
4083    List<ServerName> serversAdded = new ArrayList<>(servers.size());
4084    // Place the decommission marker first.
4085    String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;
4086    for (ServerName server : servers) {
4087      try {
4088        String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
4089        ZKUtil.createAndFailSilent(getZooKeeper(), node);
4090      } catch (KeeperException ke) {
4091        throw new HBaseIOException(
4092          this.zooKeeper.prefix("Unable to decommission '" + server.getServerName() + "'."), ke);
4093      }
4094      if (this.serverManager.addServerToDrainList(server)) {
4095        serversAdded.add(server);
4096      }
4097    }
4098    // Move the regions off the decommissioned servers.
4099    if (offload) {
4100      final List<ServerName> destServers = this.serverManager.createDestinationServersList();
4101      for (ServerName server : serversAdded) {
4102        final List<RegionInfo> regionsOnServer = this.assignmentManager.getRegionsOnServer(server);
4103        for (RegionInfo hri : regionsOnServer) {
4104          ServerName dest = balancer.randomAssignment(hri, destServers);
4105          if (dest == null) {
4106            throw new HBaseIOException("Unable to determine a plan to move " + hri);
4107          }
4108          RegionPlan rp = new RegionPlan(hri, server, dest);
4109          this.assignmentManager.moveAsync(rp);
4110        }
4111      }
4112    }
4113  }
4114
4115  /**
4116   * List region servers marked as decommissioned (previously called 'draining') to not get regions
4117   * assigned to them.
4118   * @return List of decommissioned servers.
4119   */
4120  public List<ServerName> listDecommissionedRegionServers() {
4121    return this.serverManager.getDrainingServersList();
4122  }
4123
4124  /**
4125   * Remove decommission marker (previously called 'draining') from a region server to allow regions
4126   * assignments. Load regions onto the server asynchronously if a list of regions is given
4127   * @param server Region server to remove decommission marker from.
4128   */
4129  public void recommissionRegionServer(final ServerName server,
4130    final List<byte[]> encodedRegionNames) throws IOException {
4131    // Remove the server from decommissioned (draining) server list.
4132    String parentZnode = getZooKeeper().getZNodePaths().drainingZNode;
4133    String node = ZNodePaths.joinZNode(parentZnode, server.getServerName());
4134    try {
4135      ZKUtil.deleteNodeFailSilent(getZooKeeper(), node);
4136    } catch (KeeperException ke) {
4137      throw new HBaseIOException(
4138        this.zooKeeper.prefix("Unable to recommission '" + server.getServerName() + "'."), ke);
4139    }
4140    this.serverManager.removeServerFromDrainList(server);
4141
4142    // Load the regions onto the server if we are given a list of regions.
4143    if (encodedRegionNames == null || encodedRegionNames.isEmpty()) {
4144      return;
4145    }
4146    if (!this.serverManager.isServerOnline(server)) {
4147      return;
4148    }
4149    for (byte[] encodedRegionName : encodedRegionNames) {
4150      RegionState regionState =
4151        assignmentManager.getRegionStates().getRegionState(Bytes.toString(encodedRegionName));
4152      if (regionState == null) {
4153        LOG.warn("Unknown region " + Bytes.toStringBinary(encodedRegionName));
4154        continue;
4155      }
4156      RegionInfo hri = regionState.getRegion();
4157      if (server.equals(regionState.getServerName())) {
4158        LOG.info("Skipping move of region " + hri.getRegionNameAsString()
4159          + " because region already assigned to the same server " + server + ".");
4160        continue;
4161      }
4162      RegionPlan rp = new RegionPlan(hri, regionState.getServerName(), server);
4163      this.assignmentManager.moveAsync(rp);
4164    }
4165  }
4166
4167  @Override
4168  public LockManager getLockManager() {
4169    return lockManager;
4170  }
4171
4172  public QuotaObserverChore getQuotaObserverChore() {
4173    return this.quotaObserverChore;
4174  }
4175
4176  public SpaceQuotaSnapshotNotifier getSpaceQuotaSnapshotNotifier() {
4177    return this.spaceQuotaSnapshotNotifier;
4178  }
4179
4180  @SuppressWarnings("unchecked")
4181  private RemoteProcedure<MasterProcedureEnv, ?> getRemoteProcedure(long procId) {
4182    Procedure<?> procedure = procedureExecutor.getProcedure(procId);
4183    if (procedure == null) {
4184      return null;
4185    }
4186    assert procedure instanceof RemoteProcedure;
4187    return (RemoteProcedure<MasterProcedureEnv, ?>) procedure;
4188  }
4189
4190  public void remoteProcedureCompleted(long procId) {
4191    LOG.debug("Remote procedure done, pid={}", procId);
4192    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
4193    if (procedure != null) {
4194      procedure.remoteOperationCompleted(procedureExecutor.getEnvironment());
4195    }
4196  }
4197
4198  public void remoteProcedureFailed(long procId, RemoteProcedureException error) {
4199    LOG.debug("Remote procedure failed, pid={}", procId, error);
4200    RemoteProcedure<MasterProcedureEnv, ?> procedure = getRemoteProcedure(procId);
4201    if (procedure != null) {
4202      procedure.remoteOperationFailed(procedureExecutor.getEnvironment(), error);
4203    }
4204  }
4205
4206  /**
4207   * Reopen regions provided in the argument
4208   * @param tableName   The current table name
4209   * @param regionNames The region names of the regions to reopen
4210   * @param nonceGroup  Identifier for the source of the request, a client or process
4211   * @param nonce       A unique identifier for this operation from the client or process identified
4212   *                    by <code>nonceGroup</code> (the source must ensure each operation gets a
4213   *                    unique id).
4214   * @return procedure Id
4215   * @throws IOException if reopening region fails while running procedure
4216   */
4217  long reopenRegions(final TableName tableName, final List<byte[]> regionNames,
4218    final long nonceGroup, final long nonce) throws IOException {
4219
4220    return MasterProcedureUtil
4221      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
4222
4223        @Override
4224        protected void run() throws IOException {
4225          submitProcedure(new ReopenTableRegionsProcedure(tableName, regionNames));
4226        }
4227
4228        @Override
4229        protected String getDescription() {
4230          return "ReopenTableRegionsProcedure";
4231        }
4232
4233      });
4234
4235  }
4236
4237  @Override
4238  public ReplicationPeerManager getReplicationPeerManager() {
4239    return replicationPeerManager;
4240  }
4241
4242  @Override
4243  public ReplicationLogCleanerBarrier getReplicationLogCleanerBarrier() {
4244    return replicationLogCleanerBarrier;
4245  }
4246
4247  @Override
4248  public Semaphore getSyncReplicationPeerLock() {
4249    return syncReplicationPeerLock;
4250  }
4251
4252  public HashMap<String, List<Pair<ServerName, ReplicationLoadSource>>>
4253    getReplicationLoad(ServerName[] serverNames) {
4254    List<ReplicationPeerDescription> peerList = this.getReplicationPeerManager().listPeers(null);
4255    if (peerList == null) {
4256      return null;
4257    }
4258    HashMap<String, List<Pair<ServerName, ReplicationLoadSource>>> replicationLoadSourceMap =
4259      new HashMap<>(peerList.size());
4260    peerList.stream()
4261      .forEach(peer -> replicationLoadSourceMap.put(peer.getPeerId(), new ArrayList<>()));
4262    for (ServerName serverName : serverNames) {
4263      List<ReplicationLoadSource> replicationLoadSources =
4264        getServerManager().getLoad(serverName).getReplicationLoadSourceList();
4265      for (ReplicationLoadSource replicationLoadSource : replicationLoadSources) {
4266        List<Pair<ServerName, ReplicationLoadSource>> replicationLoadSourceList =
4267          replicationLoadSourceMap.get(replicationLoadSource.getPeerID());
4268        if (replicationLoadSourceList == null) {
4269          LOG.debug("{} does not exist, but it exists "
4270            + "in znode(/hbase/replication/rs). when the rs restarts, peerId is deleted, so "
4271            + "we just need to ignore it", replicationLoadSource.getPeerID());
4272          continue;
4273        }
4274        replicationLoadSourceList.add(new Pair<>(serverName, replicationLoadSource));
4275      }
4276    }
4277    for (List<Pair<ServerName, ReplicationLoadSource>> loads : replicationLoadSourceMap.values()) {
4278      if (loads.size() > 0) {
4279        loads.sort(Comparator.comparingLong(load -> (-1) * load.getSecond().getReplicationLag()));
4280      }
4281    }
4282    return replicationLoadSourceMap;
4283  }
4284
4285  /**
4286   * This method modifies the master's configuration in order to inject replication-related features
4287   */
4288  @InterfaceAudience.Private
4289  public static void decorateMasterConfiguration(Configuration conf) {
4290    String plugins = conf.get(HBASE_MASTER_LOGCLEANER_PLUGINS);
4291    String cleanerClass = ReplicationLogCleaner.class.getCanonicalName();
4292    if (plugins == null || !plugins.contains(cleanerClass)) {
4293      conf.set(HBASE_MASTER_LOGCLEANER_PLUGINS, plugins + "," + cleanerClass);
4294    }
4295    if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
4296      plugins = conf.get(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS);
4297      cleanerClass = ReplicationHFileCleaner.class.getCanonicalName();
4298      if (!plugins.contains(cleanerClass)) {
4299        conf.set(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS, plugins + "," + cleanerClass);
4300      }
4301    }
4302  }
4303
4304  public SnapshotQuotaObserverChore getSnapshotQuotaObserverChore() {
4305    return this.snapshotQuotaChore;
4306  }
4307
4308  public ActiveMasterManager getActiveMasterManager() {
4309    return activeMasterManager;
4310  }
4311
4312  @Override
4313  public SyncReplicationReplayWALManager getSyncReplicationReplayWALManager() {
4314    return this.syncReplicationReplayWALManager;
4315  }
4316
4317  @Override
4318  public HbckChore getHbckChore() {
4319    return this.hbckChore;
4320  }
4321
4322  @Override
4323  public void runReplicationBarrierCleaner() {
4324    ReplicationBarrierCleaner rbc = this.replicationBarrierCleaner;
4325    if (rbc != null) {
4326      rbc.chore();
4327    }
4328  }
4329
4330  @Override
4331  public RSGroupInfoManager getRSGroupInfoManager() {
4332    return rsGroupInfoManager;
4333  }
4334
4335  /**
4336   * Get the compaction state of the table
4337   * @param tableName The table name
4338   * @return CompactionState Compaction state of the table
4339   */
4340  public CompactionState getCompactionState(final TableName tableName) {
4341    CompactionState compactionState = CompactionState.NONE;
4342    try {
4343      List<RegionInfo> regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);
4344      for (RegionInfo regionInfo : regions) {
4345        ServerName serverName =
4346          assignmentManager.getRegionStates().getRegionServerOfRegion(regionInfo);
4347        if (serverName == null) {
4348          continue;
4349        }
4350        ServerMetrics sl = serverManager.getLoad(serverName);
4351        if (sl == null) {
4352          continue;
4353        }
4354        RegionMetrics regionMetrics = sl.getRegionMetrics().get(regionInfo.getRegionName());
4355        if (regionMetrics == null) {
4356          LOG.warn("Can not get compaction details for the region: {} , it may be not online.",
4357            regionInfo.getRegionNameAsString());
4358          continue;
4359        }
4360        if (regionMetrics.getCompactionState() == CompactionState.MAJOR) {
4361          if (compactionState == CompactionState.MINOR) {
4362            compactionState = CompactionState.MAJOR_AND_MINOR;
4363          } else {
4364            compactionState = CompactionState.MAJOR;
4365          }
4366        } else if (regionMetrics.getCompactionState() == CompactionState.MINOR) {
4367          if (compactionState == CompactionState.MAJOR) {
4368            compactionState = CompactionState.MAJOR_AND_MINOR;
4369          } else {
4370            compactionState = CompactionState.MINOR;
4371          }
4372        }
4373      }
4374    } catch (Exception e) {
4375      compactionState = null;
4376      LOG.error("Exception when get compaction state for " + tableName.getNameAsString(), e);
4377    }
4378    return compactionState;
4379  }
4380
4381  @Override
4382  public MetaLocationSyncer getMetaLocationSyncer() {
4383    return metaLocationSyncer;
4384  }
4385
4386  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4387      allowedOnPath = ".*/src/test/.*")
4388  public MasterRegion getMasterRegion() {
4389    return masterRegion;
4390  }
4391
4392  @Override
4393  public void onConfigurationChange(Configuration newConf) {
4394    try {
4395      Superusers.initialize(newConf);
4396    } catch (IOException e) {
4397      LOG.warn("Failed to initialize SuperUsers on reloading of the configuration");
4398    }
4399    // append the quotas observer back to the master coprocessor key
4400    setQuotasObserver(newConf);
4401    // update region server coprocessor if the configuration has changed.
4402    if (
4403      CoprocessorConfigurationUtil.checkConfigurationChange(getConfiguration(), newConf,
4404        CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY) && !maintenanceMode
4405    ) {
4406      LOG.info("Update the master coprocessor(s) because the configuration has changed");
4407      initializeCoprocessorHost(newConf);
4408    }
4409  }
4410
4411  @Override
4412  protected NamedQueueRecorder createNamedQueueRecord() {
4413    final boolean isBalancerDecisionRecording =
4414      conf.getBoolean(BaseLoadBalancer.BALANCER_DECISION_BUFFER_ENABLED,
4415        BaseLoadBalancer.DEFAULT_BALANCER_DECISION_BUFFER_ENABLED);
4416    final boolean isBalancerRejectionRecording =
4417      conf.getBoolean(BaseLoadBalancer.BALANCER_REJECTION_BUFFER_ENABLED,
4418        BaseLoadBalancer.DEFAULT_BALANCER_REJECTION_BUFFER_ENABLED);
4419    if (isBalancerDecisionRecording || isBalancerRejectionRecording) {
4420      return NamedQueueRecorder.getInstance(conf);
4421    } else {
4422      return null;
4423    }
4424  }
4425
4426  @Override
4427  protected boolean clusterMode() {
4428    return true;
4429  }
4430
4431  public String getClusterId() {
4432    if (activeMaster) {
4433      return clusterId;
4434    }
4435    return cachedClusterId.getFromCacheOrFetch();
4436  }
4437
4438  public Optional<ServerName> getActiveMaster() {
4439    return activeMasterManager.getActiveMasterServerName();
4440  }
4441
4442  public List<ServerName> getBackupMasters() {
4443    return activeMasterManager.getBackupMasters();
4444  }
4445
4446  @Override
4447  public Iterator<ServerName> getBootstrapNodes() {
4448    return regionServerTracker.getRegionServers().iterator();
4449  }
4450
4451  @Override
4452  public List<HRegionLocation> getMetaLocations() {
4453    return metaRegionLocationCache.getMetaRegionLocations();
4454  }
4455
4456  @Override
4457  public void flushMasterStore() throws IOException {
4458    LOG.info("Force flush master local region.");
4459    if (this.cpHost != null) {
4460      try {
4461        cpHost.preMasterStoreFlush();
4462      } catch (IOException ioe) {
4463        LOG.error("Error invoking master coprocessor preMasterStoreFlush()", ioe);
4464      }
4465    }
4466    masterRegion.flush(true);
4467    if (this.cpHost != null) {
4468      try {
4469        cpHost.postMasterStoreFlush();
4470      } catch (IOException ioe) {
4471        LOG.error("Error invoking master coprocessor postMasterStoreFlush()", ioe);
4472      }
4473    }
4474  }
4475
4476  public Collection<ServerName> getLiveRegionServers() {
4477    return regionServerTracker.getRegionServers();
4478  }
4479
4480  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4481      allowedOnPath = ".*/src/test/.*")
4482  void setLoadBalancer(RSGroupBasedLoadBalancer loadBalancer) {
4483    this.balancer = loadBalancer;
4484  }
4485
4486  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4487      allowedOnPath = ".*/src/test/.*")
4488  void setAssignmentManager(AssignmentManager assignmentManager) {
4489    this.assignmentManager = assignmentManager;
4490  }
4491
4492  @RestrictedApi(explanation = "Should only be called in tests", link = "",
4493      allowedOnPath = ".*/src/test/.*")
4494  static void setDisableBalancerChoreForTest(boolean disable) {
4495    disableBalancerChoreForTest = disable;
4496  }
4497
4498  private void setQuotasObserver(Configuration conf) {
4499    // Add the Observer to delete quotas on table deletion before starting all CPs by
4500    // default with quota support, avoiding if user specifically asks to not load this Observer.
4501    if (QuotaUtil.isQuotaEnabled(conf)) {
4502      updateConfigurationForQuotasObserver(conf);
4503    }
4504  }
4505
4506  private void initializeCoprocessorHost(Configuration conf) {
4507    // initialize master side coprocessors before we start handling requests
4508    this.cpHost = new MasterCoprocessorHost(this, conf);
4509  }
4510
4511  @Override
4512  public long flushTable(TableName tableName, List<byte[]> columnFamilies, long nonceGroup,
4513    long nonce) throws IOException {
4514    checkInitialized();
4515
4516    if (
4517      !getConfiguration().getBoolean(MasterFlushTableProcedureManager.FLUSH_PROCEDURE_ENABLED,
4518        MasterFlushTableProcedureManager.FLUSH_PROCEDURE_ENABLED_DEFAULT)
4519    ) {
4520      throw new DoNotRetryIOException("FlushTableProcedureV2 is DISABLED");
4521    }
4522
4523    return MasterProcedureUtil
4524      .submitProcedure(new MasterProcedureUtil.NonceProcedureRunnable(this, nonceGroup, nonce) {
4525        @Override
4526        protected void run() throws IOException {
4527          getMaster().getMasterCoprocessorHost().preTableFlush(tableName);
4528          LOG.info(getClientIdAuditPrefix() + " flush " + tableName);
4529          submitProcedure(
4530            new FlushTableProcedure(procedureExecutor.getEnvironment(), tableName, columnFamilies));
4531          getMaster().getMasterCoprocessorHost().postTableFlush(tableName);
4532        }
4533
4534        @Override
4535        protected String getDescription() {
4536          return "FlushTableProcedure";
4537        }
4538      });
4539  }
4540}