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