Skip to content

Commit

Permalink
HBASE-21521 Expose master startup status via web UI (apache#4788)
Browse files Browse the repository at this point in the history
Signed-off-by: Bryan Beaudreault <bbeaudreault@apache.org>
  • Loading branch information
sunhelly committed Feb 13, 2023
1 parent d401913 commit 9faeee4
Show file tree
Hide file tree
Showing 13 changed files with 367 additions and 48 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
<%if HBaseConfiguration.isShowConfInServlet()%>
<li><a href="/conf">HBase Configuration</a></li>
</%if>
<li><a href="/startupProgress.jsp">Startup Progress</a></li>
</ul>
</div><!--/.nav-collapse -->
</div>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hadoop.hbase.ZNodeClearer;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskGroup;
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
import org.apache.hadoop.hbase.zookeeper.ZKListener;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
Expand Down Expand Up @@ -218,16 +219,18 @@ private void handleMasterNodeChange() {
* Block until becoming the active master. Method blocks until there is not another active master
* and our attempt to become the new active master is successful. This also makes sure that we are
* watching the master znode so will be notified if another master dies.
* @param checkInterval the interval to check if the master is stopped
* @param startupStatus the monitor status to track the progress
* @param checkInterval the interval to check if the master is stopped
* @param startupTaskGroup the task group for master startup to track the progress
* @return True if no issue becoming active master else false if another master was running or if
* some other problem (zookeeper, stop flag has been set on this Master)
*/
boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupStatus) {
boolean blockUntilBecomingActiveMaster(int checkInterval, TaskGroup startupTaskGroup) {
MonitoredTask blockUntilActive =
startupTaskGroup.addTask("Blocking until becoming active master");
String backupZNode = ZNodePaths
.joinZNode(this.watcher.getZNodePaths().backupMasterAddressesZNode, this.sn.toString());
while (!(master.isAborted() || master.isStopped())) {
startupStatus.setStatus("Trying to register in ZK as active master");
blockUntilActive.setStatus("Trying to register in ZK as active master");
// Try to become the active master, watch if there is another master.
// Write out our ServerName as versioned bytes.
try {
Expand All @@ -246,7 +249,7 @@ boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupS
ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());

// We are the master, return
startupStatus.setStatus("Successfully registered as active master.");
blockUntilActive.setStatus("Successfully registered as active master.");
this.clusterHasActiveMaster.set(true);
activeMasterServerName = sn;
LOG.info("Registered as active master=" + this.sn);
Expand Down Expand Up @@ -291,7 +294,7 @@ boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupS
}
}
LOG.info(msg);
startupStatus.setStatus(msg);
blockUntilActive.setStatus(msg);
} catch (KeeperException ke) {
master.abort("Received an unexpected KeeperException, aborting", ke);
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,7 @@
import org.apache.hadoop.hbase.mob.MobFileCompactionChore;
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
import org.apache.hadoop.hbase.monitoring.TaskGroup;
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
Expand Down Expand Up @@ -444,6 +445,8 @@ public class HMaster extends HRegionServer implements MasterServices {
public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move";
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true;

private TaskGroup startupTaskGroup;

/**
* Initializes the HMaster. The steps are as follows:
* <p>
Expand All @@ -452,9 +455,8 @@ public class HMaster extends HRegionServer implements MasterServices {
* <li>Start the ActiveMasterManager.
* </ol>
* <p>
* Remaining steps of initialization occur in
* {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
* one.
* Remaining steps of initialization occur in {@link #finishActiveMasterInitialization()} after
* the master becomes the active one.
*/
public HMaster(final Configuration conf) throws IOException {
super(conf);
Expand Down Expand Up @@ -887,12 +889,13 @@ private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperEx
* Notice that now we will not schedule a special procedure to make meta online(unless the first
* time where meta has not been created yet), we will rely on SCP to bring meta online.
*/
private void finishActiveMasterInitialization(MonitoredTask status)

private void finishActiveMasterInitialization()
throws IOException, InterruptedException, KeeperException, ReplicationException {
/*
* We are active master now... go initialize components we need to run.
*/
status.setStatus("Initializing Master file system");
startupTaskGroup.addTask("Initializing Master file system");

this.masterActiveTime = EnvironmentEdgeManager.currentTime();
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
Expand All @@ -905,15 +908,15 @@ private void finishActiveMasterInitialization(MonitoredTask status)

// warm-up HTDs cache on master initialization
if (preLoadTableDescriptors) {
status.setStatus("Pre-loading table descriptors");
startupTaskGroup.addTask("Pre-loading table descriptors");
this.tableDescriptors.getAll();
}

// Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
// only after it has checked in with the Master. At least a few tests ask Master for clusterId
// before it has called its run method and before RegionServer has done the reportForDuty.
ClusterId clusterId = fileSystemManager.getClusterId();
status.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
startupTaskGroup.addTask("Publishing Cluster ID " + clusterId + " in ZooKeeper");
ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
this.clusterId = clusterId.toString();

Expand All @@ -932,7 +935,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
}

status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
startupTaskGroup.addTask("Initialize ServerManager and schedule SCP for crash servers");
// The below two managers must be created before loading procedures, as they will be used during
// loading.
// initialize master local region
Expand Down Expand Up @@ -982,8 +985,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
? new MirroringTableStateManager(this)
: new TableStateManager(this);

status.setStatus("Initializing ZK system trackers");
startupTaskGroup.addTask("Initializing ZK system trackers");
initializeZKBasedSystemTrackers();

// Set ourselves as active Master now our claim has succeeded up in zk.
this.activeMaster = true;

Expand All @@ -995,19 +999,19 @@ private void finishActiveMasterInitialization(MonitoredTask status)

// This is for backwards compatibility
// See HBASE-11393
status.setStatus("Update TableCFs node in ZNode");
startupTaskGroup.addTask("Update TableCFs node in ZNode");
ReplicationPeerConfigUpgrader tableCFsUpdater =
new ReplicationPeerConfigUpgrader(zooKeeper, conf);
tableCFsUpdater.copyTableCFs();

if (!maintenanceMode) {
status.setStatus("Initializing master coprocessors");
startupTaskGroup.addTask("Initializing master coprocessors");
setQuotasObserver(conf);
initializeCoprocessorHost(conf);
}

// Checking if meta needs initializing.
status.setStatus("Initializing meta table if this is a new deploy");
startupTaskGroup.addTask("Initializing meta table if this is a new deploy");
InitMetaProcedure initMetaProc = null;
// Print out state of hbase:meta on startup; helps debugging.
if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) {
Expand All @@ -1030,7 +1034,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());

// start up all service threads.
status.setStatus("Initializing master service threads");
startupTaskGroup.addTask("Initializing master service threads");
startServiceThreads();
// wait meta to be initialized after we start procedure executor
if (initMetaProc != null) {
Expand All @@ -1046,16 +1050,16 @@ private void finishActiveMasterInitialization(MonitoredTask status)
// With this as part of master initialization, it precludes our being able to start a single
// server that is both Master and RegionServer. Needs more thought. TODO.
String statusStr = "Wait for region servers to report in";
status.setStatus(statusStr);
LOG.info(Objects.toString(status));
waitForRegionServers(status);
MonitoredTask waitRegionServer = startupTaskGroup.addTask(statusStr);
LOG.info(Objects.toString(waitRegionServer));
waitForRegionServers(waitRegionServer);

// Check if master is shutting down because issue initializing regionservers or balancer.
if (isStopped()) {
return;
}

status.setStatus("Starting assignment manager");
startupTaskGroup.addTask("Starting assignment manager");
// FIRST HBASE:META READ!!!!
// The below cannot make progress w/o hbase:meta being online.
// This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
Expand Down Expand Up @@ -1132,7 +1136,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());

// Start balancer and meta catalog janitor after meta and regions have been assigned.
status.setStatus("Starting balancer and catalog janitor");
startupTaskGroup.addTask("Starting balancer and catalog janitor");
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
getChoreService().scheduleChore(clusterStatusChore);
this.balancerChore = new BalancerChore(this);
Expand All @@ -1154,7 +1158,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
if (!waitForNamespaceOnline()) {
return;
}
status.setStatus("Starting cluster schema service");
startupTaskGroup.addTask("Starting cluster schema service");
try {
initClusterSchemaService();
} catch (IllegalStateException e) {
Expand All @@ -1177,7 +1181,6 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
}

status.markComplete("Initialization successful");
LOG.info(String.format("Master has completed initialization %.3fsec",
(EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));
this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();
Expand All @@ -1196,6 +1199,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
// Set master as 'initialized'.
setInitialized(true);
startupTaskGroup.markComplete("Initialization successful");
MonitoredTask status =
TaskMonitor.get().createStatus("Progress after master initialized", false, true);

if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
// create missing CFs in meta table after master is set to 'initialized'.
Expand Down Expand Up @@ -1277,9 +1283,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
LOG.debug("Balancer post startup initialization complete, took "
+ ((EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds");
}

this.rollingUpgradeChore = new RollingUpgradeChore(this);
getChoreService().scheduleChore(rollingUpgradeChore);
status.markComplete("Progress after master initialized complete");
}

private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
Expand Down Expand Up @@ -2387,14 +2393,19 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
Threads.sleep(timeout);
}
}
MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
status.setDescription("Master startup");

// Here for the master startup process, we use TaskGroup to monitor the whole progress.
// The UI is similar to how Hadoop designed the startup page for the NameNode.
// See HBASE-21521 for more details.
// We do not cleanup the startupTaskGroup, let the startup progress information
// be permanent in the MEM.
startupTaskGroup = TaskMonitor.createTaskGroup(true, "Master startup");
try {
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
finishActiveMasterInitialization(status);
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
finishActiveMasterInitialization();
}
} catch (Throwable t) {
status.setStatus("Failed to become active: " + t.getMessage());
startupTaskGroup.abort("Failed to become active master due to:" + t.getMessage());
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
if (
Expand All @@ -2408,8 +2419,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
} else {
abort("Unhandled exception. Starting shutdown.", t);
}
} finally {
status.cleanup();
}
}

Expand Down Expand Up @@ -3099,6 +3108,10 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
return rsFatals;
}

public TaskGroup getStartupProgress() {
return startupTaskGroup;
}

/**
* Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices ma
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, workingDirFs);
// update the running tasks
this.status = TaskMonitor.get()
.createStatus("Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, true);
.createStatus("Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, false, false);
this.snapshotManifest =
SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor, status);
}
Expand Down
Loading

0 comments on commit 9faeee4

Please sign in to comment.