Skip to content

Commit 4b9214c

Browse files
authored
HBASE-21521 Expose master startup status via web UI (#4788) (#5021) (#5026)
Signed-off-by: Bryan Beaudreault <bbeaudreault@apache.org>
1 parent ee618d5 commit 4b9214c

File tree

12 files changed

+367
-47
lines changed

12 files changed

+367
-47
lines changed

hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -177,6 +177,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
177177
<%if HBaseConfiguration.isShowConfInServlet()%>
178178
<li><a href="/conf">HBase Configuration</a></li>
179179
</%if>
180+
<li><a href="/startupProgress.jsp">Startup Progress</a></li>
180181
</ul>
181182
</div><!--/.nav-collapse -->
182183
</div>

hbase-server/src/main/java/org/apache/hadoop/hbase/master/ActiveMasterManager.java

Lines changed: 9 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import org.apache.hadoop.hbase.ZNodeClearer;
2828
import org.apache.hadoop.hbase.exceptions.DeserializationException;
2929
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
30+
import org.apache.hadoop.hbase.monitoring.TaskGroup;
3031
import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
3132
import org.apache.hadoop.hbase.zookeeper.ZKListener;
3233
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -218,16 +219,18 @@ private void handleMasterNodeChange() {
218219
* Block until becoming the active master. Method blocks until there is not another active master
219220
* and our attempt to become the new active master is successful. This also makes sure that we are
220221
* watching the master znode so will be notified if another master dies.
221-
* @param checkInterval the interval to check if the master is stopped
222-
* @param startupStatus the monitor status to track the progress
222+
* @param checkInterval the interval to check if the master is stopped
223+
* @param startupTaskGroup the task group for master startup to track the progress
223224
* @return True if no issue becoming active master else false if another master was running or if
224225
* some other problem (zookeeper, stop flag has been set on this Master)
225226
*/
226-
boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupStatus) {
227+
boolean blockUntilBecomingActiveMaster(int checkInterval, TaskGroup startupTaskGroup) {
228+
MonitoredTask blockUntilActive =
229+
startupTaskGroup.addTask("Blocking until becoming active master");
227230
String backupZNode = ZNodePaths
228231
.joinZNode(this.watcher.getZNodePaths().backupMasterAddressesZNode, this.sn.toString());
229232
while (!(master.isAborted() || master.isStopped())) {
230-
startupStatus.setStatus("Trying to register in ZK as active master");
233+
blockUntilActive.setStatus("Trying to register in ZK as active master");
231234
// Try to become the active master, watch if there is another master.
232235
// Write out our ServerName as versioned bytes.
233236
try {
@@ -246,7 +249,7 @@ boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupS
246249
ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());
247250

248251
// We are the master, return
249-
startupStatus.setStatus("Successfully registered as active master.");
252+
blockUntilActive.setStatus("Successfully registered as active master.");
250253
this.clusterHasActiveMaster.set(true);
251254
activeMasterServerName = sn;
252255
LOG.info("Registered as active master=" + this.sn);
@@ -291,7 +294,7 @@ boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupS
291294
}
292295
}
293296
LOG.info(msg);
294-
startupStatus.setStatus(msg);
297+
blockUntilActive.setStatus(msg);
295298
} catch (KeeperException ke) {
296299
master.abort("Received an unexpected KeeperException, aborting", ke);
297300
return false;

hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java

Lines changed: 41 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -183,6 +183,7 @@
183183
import org.apache.hadoop.hbase.mob.MobFileCompactionChore;
184184
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
185185
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
186+
import org.apache.hadoop.hbase.monitoring.TaskGroup;
186187
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
187188
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
188189
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
@@ -447,6 +448,8 @@ public class HMaster extends HRegionServer implements MasterServices {
447448
public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move";
448449
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true;
449450

451+
private TaskGroup startupTaskGroup;
452+
450453
/**
451454
* Initializes the HMaster. The steps are as follows:
452455
* <p>
@@ -455,9 +458,8 @@ public class HMaster extends HRegionServer implements MasterServices {
455458
* <li>Start the ActiveMasterManager.
456459
* </ol>
457460
* <p>
458-
* Remaining steps of initialization occur in
459-
* {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
460-
* one.
461+
* Remaining steps of initialization occur in {@link #finishActiveMasterInitialization()} after
462+
* the master becomes the active one.
461463
*/
462464
public HMaster(final Configuration conf) throws IOException {
463465
super(conf);
@@ -887,12 +889,12 @@ private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperEx
887889
* Notice that now we will not schedule a special procedure to make meta online(unless the first
888890
* time where meta has not been created yet), we will rely on SCP to bring meta online.
889891
*/
890-
private void finishActiveMasterInitialization(MonitoredTask status) throws IOException,
891-
InterruptedException, KeeperException, ReplicationException, DeserializationException {
892+
private void finishActiveMasterInitialization() throws IOException, InterruptedException,
893+
KeeperException, ReplicationException, DeserializationException {
892894
/*
893895
* We are active master now... go initialize components we need to run.
894896
*/
895-
status.setStatus("Initializing Master file system");
897+
startupTaskGroup.addTask("Initializing Master file system");
896898

897899
this.masterActiveTime = EnvironmentEdgeManager.currentTime();
898900
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -905,15 +907,15 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
905907

906908
// warm-up HTDs cache on master initialization
907909
if (preLoadTableDescriptors) {
908-
status.setStatus("Pre-loading table descriptors");
910+
startupTaskGroup.addTask("Pre-loading table descriptors");
909911
this.tableDescriptors.getAll();
910912
}
911913

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

@@ -932,7 +934,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
932934
}
933935
}
934936

935-
status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
937+
startupTaskGroup.addTask("Initialize ServerManager and schedule SCP for crash servers");
936938
// The below two managers must be created before loading procedures, as they will be used during
937939
// loading.
938940
// initialize master local region
@@ -982,8 +984,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
982984
? new MirroringTableStateManager(this)
983985
: new TableStateManager(this);
984986

985-
status.setStatus("Initializing ZK system trackers");
987+
startupTaskGroup.addTask("Initializing ZK system trackers");
986988
initializeZKBasedSystemTrackers();
989+
987990
// Set ourselves as active Master now our claim has succeeded up in zk.
988991
this.activeMaster = true;
989992

@@ -995,19 +998,19 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
995998

996999
// This is for backwards compatibility
9971000
// See HBASE-11393
998-
status.setStatus("Update TableCFs node in ZNode");
1001+
startupTaskGroup.addTask("Update TableCFs node in ZNode");
9991002
ReplicationPeerConfigUpgrader tableCFsUpdater =
10001003
new ReplicationPeerConfigUpgrader(zooKeeper, conf);
10011004
tableCFsUpdater.copyTableCFs();
10021005

10031006
if (!maintenanceMode) {
1004-
status.setStatus("Initializing master coprocessors");
1007+
startupTaskGroup.addTask("Initializing master coprocessors");
10051008
setQuotasObserver(conf);
10061009
initializeCoprocessorHost(conf);
10071010
}
10081011

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

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

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

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

11341137
// Start balancer and meta catalog janitor after meta and regions have been assigned.
1135-
status.setStatus("Starting balancer and catalog janitor");
1138+
startupTaskGroup.addTask("Starting balancer and catalog janitor");
11361139
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
11371140
getChoreService().scheduleChore(clusterStatusChore);
11381141
this.balancerChore = new BalancerChore(this);
@@ -1154,7 +1157,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
11541157
if (!waitForNamespaceOnline()) {
11551158
return;
11561159
}
1157-
status.setStatus("Starting cluster schema service");
1160+
startupTaskGroup.addTask("Starting cluster schema service");
11581161
try {
11591162
initClusterSchemaService();
11601163
} catch (IllegalStateException e) {
@@ -1177,7 +1180,6 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
11771180
}
11781181
}
11791182

1180-
status.markComplete("Initialization successful");
11811183
LOG.info(String.format("Master has completed initialization %.3fsec",
11821184
(EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));
11831185
this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();
@@ -1196,6 +1198,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
11961198
}
11971199
// Set master as 'initialized'.
11981200
setInitialized(true);
1201+
startupTaskGroup.markComplete("Initialization successful");
1202+
MonitoredTask status =
1203+
TaskMonitor.get().createStatus("Progress after master initialized", false, true);
11991204

12001205
if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
12011206
// create missing CFs in meta table after master is set to 'initialized'.
@@ -1284,6 +1289,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
12841289

12851290
this.rollingUpgradeChore = new RollingUpgradeChore(this);
12861291
getChoreService().scheduleChore(rollingUpgradeChore);
1292+
status.markComplete("Progress after master initialized complete");
12871293
}
12881294

12891295
private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
@@ -2385,14 +2391,19 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
23852391
Threads.sleep(timeout);
23862392
}
23872393
}
2388-
MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
2389-
status.setDescription("Master startup");
2394+
2395+
// Here for the master startup process, we use TaskGroup to monitor the whole progress.
2396+
// The UI is similar to how Hadoop designed the startup page for the NameNode.
2397+
// See HBASE-21521 for more details.
2398+
// We do not cleanup the startupTaskGroup, let the startup progress information
2399+
// be permanent in the MEM.
2400+
startupTaskGroup = TaskMonitor.createTaskGroup(true, "Master startup");
23902401
try {
2391-
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
2392-
finishActiveMasterInitialization(status);
2402+
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
2403+
finishActiveMasterInitialization();
23932404
}
23942405
} catch (Throwable t) {
2395-
status.setStatus("Failed to become active: " + t.getMessage());
2406+
startupTaskGroup.abort("Failed to become active master due to:" + t.getMessage());
23962407
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
23972408
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
23982409
if (
@@ -2406,8 +2417,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
24062417
} else {
24072418
abort("Unhandled exception. Starting shutdown.", t);
24082419
}
2409-
} finally {
2410-
status.cleanup();
24112420
}
24122421
}
24132422

@@ -3097,6 +3106,10 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
30973106
return rsFatals;
30983107
}
30993108

3109+
public TaskGroup getStartupProgress() {
3110+
return startupTaskGroup;
3111+
}
3112+
31003113
/**
31013114
* Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
31023115
*/

hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -125,8 +125,8 @@ public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices ma
125125
// prepare the verify
126126
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, workingDirFs);
127127
// update the running tasks
128-
this.status = TaskMonitor.get()
129-
.createStatus("Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, true);
128+
this.status = TaskMonitor.get().createStatus(
129+
"Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, false, true);
130130
this.snapshotManifest =
131131
SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor, status);
132132
}

0 commit comments

Comments
 (0)