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