Skip to content

Commit 9faeee4

Browse files
committed
HBASE-21521 Expose master startup status via web UI (apache#4788)
Signed-off-by: Bryan Beaudreault <[email protected]>
1 parent d401913 commit 9faeee4

File tree

13 files changed

+367
-48
lines changed

13 files changed

+367
-48
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
@@ -167,6 +167,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager();
167167
<%if HBaseConfiguration.isShowConfInServlet()%>
168168
<li><a href="/conf">HBase Configuration</a></li>
169169
</%if>
170+
<li><a href="/startupProgress.jsp">Startup Progress</a></li>
170171
</ul>
171172
</div><!--/.nav-collapse -->
172173
</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
@@ -178,6 +178,7 @@
178178
import org.apache.hadoop.hbase.mob.MobFileCompactionChore;
179179
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
180180
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
181+
import org.apache.hadoop.hbase.monitoring.TaskGroup;
181182
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
182183
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
183184
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
@@ -444,6 +445,8 @@ public class HMaster extends HRegionServer implements MasterServices {
444445
public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move";
445446
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true;
446447

448+
private TaskGroup startupTaskGroup;
449+
447450
/**
448451
* Initializes the HMaster. The steps are as follows:
449452
* <p>
@@ -452,9 +455,8 @@ public class HMaster extends HRegionServer implements MasterServices {
452455
* <li>Start the ActiveMasterManager.
453456
* </ol>
454457
* <p>
455-
* Remaining steps of initialization occur in
456-
* {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
457-
* one.
458+
* Remaining steps of initialization occur in {@link #finishActiveMasterInitialization()} after
459+
* the master becomes the active one.
458460
*/
459461
public HMaster(final Configuration conf) throws IOException {
460462
super(conf);
@@ -887,12 +889,13 @@ 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)
892+
893+
private void finishActiveMasterInitialization()
891894
throws IOException, InterruptedException, KeeperException, ReplicationException {
892895
/*
893896
* We are active master now... go initialize components we need to run.
894897
*/
895-
status.setStatus("Initializing Master file system");
898+
startupTaskGroup.addTask("Initializing Master file system");
896899

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

906909
// warm-up HTDs cache on master initialization
907910
if (preLoadTableDescriptors) {
908-
status.setStatus("Pre-loading table descriptors");
911+
startupTaskGroup.addTask("Pre-loading table descriptors");
909912
this.tableDescriptors.getAll();
910913
}
911914

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

@@ -932,7 +935,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
932935
}
933936
}
934937

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

985-
status.setStatus("Initializing ZK system trackers");
988+
startupTaskGroup.addTask("Initializing ZK system trackers");
986989
initializeZKBasedSystemTrackers();
990+
987991
// Set ourselves as active Master now our claim has succeeded up in zk.
988992
this.activeMaster = true;
989993

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

9961000
// This is for backwards compatibility
9971001
// See HBASE-11393
998-
status.setStatus("Update TableCFs node in ZNode");
1002+
startupTaskGroup.addTask("Update TableCFs node in ZNode");
9991003
ReplicationPeerConfigUpgrader tableCFsUpdater =
10001004
new ReplicationPeerConfigUpgrader(zooKeeper, conf);
10011005
tableCFsUpdater.copyTableCFs();
10021006

10031007
if (!maintenanceMode) {
1004-
status.setStatus("Initializing master coprocessors");
1008+
startupTaskGroup.addTask("Initializing master coprocessors");
10051009
setQuotasObserver(conf);
10061010
initializeCoprocessorHost(conf);
10071011
}
10081012

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

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

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

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

11341138
// Start balancer and meta catalog janitor after meta and regions have been assigned.
1135-
status.setStatus("Starting balancer and catalog janitor");
1139+
startupTaskGroup.addTask("Starting balancer and catalog janitor");
11361140
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
11371141
getChoreService().scheduleChore(clusterStatusChore);
11381142
this.balancerChore = new BalancerChore(this);
@@ -1154,7 +1158,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
11541158
if (!waitForNamespaceOnline()) {
11551159
return;
11561160
}
1157-
status.setStatus("Starting cluster schema service");
1161+
startupTaskGroup.addTask("Starting cluster schema service");
11581162
try {
11591163
initClusterSchemaService();
11601164
} catch (IllegalStateException e) {
@@ -1177,7 +1181,6 @@ private void finishActiveMasterInitialization(MonitoredTask status)
11771181
}
11781182
}
11791183

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

12001206
if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
12011207
// create missing CFs in meta table after master is set to 'initialized'.
@@ -1277,9 +1283,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
12771283
LOG.debug("Balancer post startup initialization complete, took "
12781284
+ ((EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds");
12791285
}
1280-
12811286
this.rollingUpgradeChore = new RollingUpgradeChore(this);
12821287
getChoreService().scheduleChore(rollingUpgradeChore);
1288+
status.markComplete("Progress after master initialized complete");
12831289
}
12841290

12851291
private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
@@ -2387,14 +2393,19 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
23872393
Threads.sleep(timeout);
23882394
}
23892395
}
2390-
MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
2391-
status.setDescription("Master startup");
2396+
2397+
// Here for the master startup process, we use TaskGroup to monitor the whole progress.
2398+
// The UI is similar to how Hadoop designed the startup page for the NameNode.
2399+
// See HBASE-21521 for more details.
2400+
// We do not cleanup the startupTaskGroup, let the startup progress information
2401+
// be permanent in the MEM.
2402+
startupTaskGroup = TaskMonitor.createTaskGroup(true, "Master startup");
23922403
try {
2393-
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
2394-
finishActiveMasterInitialization(status);
2404+
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
2405+
finishActiveMasterInitialization();
23952406
}
23962407
} catch (Throwable t) {
2397-
status.setStatus("Failed to become active: " + t.getMessage());
2408+
startupTaskGroup.abort("Failed to become active master due to:" + t.getMessage());
23982409
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
23992410
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
24002411
if (
@@ -2408,8 +2419,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
24082419
} else {
24092420
abort("Unhandled exception. Starting shutdown.", t);
24102421
}
2411-
} finally {
2412-
status.cleanup();
24132422
}
24142423
}
24152424

@@ -3099,6 +3108,10 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
30993108
return rsFatals;
31003109
}
31013110

3111+
public TaskGroup getStartupProgress() {
3112+
return startupTaskGroup;
3113+
}
3114+
31023115
/**
31033116
* Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
31043117
*/

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -128,7 +128,7 @@ public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices ma
128128
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, workingDirFs);
129129
// update the running tasks
130130
this.status = TaskMonitor.get()
131-
.createStatus("Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, true);
131+
.createStatus("Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, false, false);
132132
this.snapshotManifest =
133133
SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor, status);
134134
}

0 commit comments

Comments
 (0)