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