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