Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HBASE-21521 Expose master startup status via web UI #4788

Merged
merged 1 commit into from
Feb 13, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -177,6 +177,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 @@ -186,6 +186,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.namequeues.NamedQueueRecorder;
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
Expand Down Expand Up @@ -465,6 +466,8 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
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 @@ -473,9 +476,8 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
* <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, "Master");
Expand Down Expand Up @@ -908,12 +910,12 @@ 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) throws IOException,
InterruptedException, KeeperException, ReplicationException, DeserializationException {
private void finishActiveMasterInitialization() throws IOException, InterruptedException,
KeeperException, ReplicationException, DeserializationException {
/*
* 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 @@ -926,15 +928,15 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc

// 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 @@ -953,7 +955,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
}
}

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 @@ -1000,9 +1002,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// This manager must be accessed AFTER hbase:meta is confirmed on line..
this.tableStateManager = new TableStateManager(this);

status.setStatus("Initializing ZK system trackers");
startupTaskGroup.addTask("Initializing ZK system trackers");
initializeZKBasedSystemTrackers();
status.setStatus("Loading last flushed sequence id of regions");
startupTaskGroup.addTask("Loading last flushed sequence id of regions");
try {
this.serverManager.loadLastFlushedSequenceIds();
} catch (IOException e) {
Expand All @@ -1018,7 +1020,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
zombieDetector.start();

if (!maintenanceMode) {
status.setStatus("Initializing master coprocessors");
startupTaskGroup.addTask("Initializing master coprocessors");
setQuotasObserver(conf);
initializeCoprocessorHost(conf);
} else {
Expand All @@ -1029,7 +1031,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
}

// 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 @@ -1049,7 +1051,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
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 @@ -1062,16 +1064,16 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// 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 @@ -1136,7 +1138,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
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 @@ -1156,7 +1158,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
if (!waitForNamespaceOnline()) {
return;
}
status.setStatus("Starting cluster schema service");
startupTaskGroup.addTask("Starting cluster schema service");
try {
initClusterSchemaService();
} catch (IllegalStateException e) {
Expand All @@ -1179,7 +1181,6 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
}
}

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

if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
// create missing CFs in meta table after master is set to 'initialized'.
Expand Down Expand Up @@ -1286,6 +1290,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc

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

private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
Expand Down Expand Up @@ -2399,14 +2404,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());
bbeaudreault marked this conversation as resolved.
Show resolved Hide resolved
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 @@ -2420,8 +2430,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
} else {
abort("Unhandled exception. Starting shutdown.", t);
}
} finally {
status.cleanup();
sunhelly marked this conversation as resolved.
Show resolved Hide resolved
}
}

Expand Down Expand Up @@ -3097,6 +3105,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
@@ -0,0 +1,115 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.monitoring;

import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.ConcurrentLinkedDeque;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* The {@link TaskGroup} can be seen as a big {@link MonitoredTask}, which contains a list of sub
* monitored tasks. The monitored tasks in the group are still be managed by the
* {@link TaskMonitor}, but whether to clear/expire the monitored tasks in a task group is optional.
* Since the monitored task already has journals, which mark the phases in a task, we still also
* need a task group to monitor a big task/process because the journals in a task is serial but the
* tasks in the task group can be parallel, then we have more flexible ability to monitor the
* process. Grouping the tasks is not strictly necessary but it is cleaner for presentation to
* operators. We might want to display the tasks in a group in a list view where each task can be
* collapsed (probably by default) or expanded.
*/
@InterfaceAudience.Private
public class TaskGroup extends MonitoredTaskImpl {
private static final Logger LOG = LoggerFactory.getLogger(TaskGroup.class);

/** Sub-tasks in the group */
private final ConcurrentLinkedDeque<MonitoredTask> tasks = new ConcurrentLinkedDeque<>();
bbeaudreault marked this conversation as resolved.
Show resolved Hide resolved

/** Whether to ignore to track(e.g. show/clear/expire) in the singleton {@link TaskMonitor} */
private boolean ignoreSubTasksInTaskMonitor;

/** Used to track this task group in {@link TaskMonitor} */
private final MonitoredTask delegate;

public TaskGroup(boolean ignoreSubTasksInTaskMonitor, String description) {
super(false, description);
this.ignoreSubTasksInTaskMonitor = ignoreSubTasksInTaskMonitor;
this.delegate = TaskMonitor.get().createStatus(description, false, true);
}

public synchronized MonitoredTask addTask(String description) {
return addTask(description, true);
}

/**
* Add a new task to the group, and before that might complete the last task in the group
* @param description the description of the new task
* @param withCompleteLast whether to complete the last task in the group
* @return the added new task
*/
public synchronized MonitoredTask addTask(String description, boolean withCompleteLast) {
if (withCompleteLast) {
MonitoredTask previousTask = this.tasks.peekLast();
if (
previousTask != null && previousTask.getState() != State.COMPLETE
&& previousTask.getState() != State.ABORTED
) {
previousTask.markComplete("Completed");
}
}
MonitoredTask task =
TaskMonitor.get().createStatus(description, ignoreSubTasksInTaskMonitor, true);
this.setStatus(description);
this.tasks.addLast(task);
delegate.setStatus(description);
return task;
}

public synchronized Collection<MonitoredTask> getTasks() {
return Collections.unmodifiableCollection(this.tasks);
}

@Override
public synchronized void abort(String msg) {
setStatus(msg);
setState(State.ABORTED);
for (MonitoredTask task : tasks) {
if (task.getState() != State.COMPLETE && task.getState() != State.ABORTED) {
task.abort(msg);
}
}
delegate.abort(msg);
}

@Override
public synchronized void markComplete(String msg) {
setState(State.COMPLETE);
setStatus(msg);
if (tasks.getLast() != null) {
tasks.getLast().markComplete(msg);
}
delegate.markComplete(msg);
}

@Override
public synchronized void cleanup() {
this.tasks.clear();
}
}
Loading