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