Skip to content

Commit 8e28152

Browse files
committed
HBASE-21521 Expose master startup status via web UI (apache#4788)
Signed-off-by: Bryan Beaudreault <bbeaudreault@apache.org>
1 parent 8df3212 commit 8e28152

File tree

12 files changed

+370
-46
lines changed

12 files changed

+370
-46
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;
@@ -200,16 +201,18 @@ private void handleMasterNodeChange() {
200201
* Block until becoming the active master. Method blocks until there is not another active master
201202
* and our attempt to become the new active master is successful. This also makes sure that we are
202203
* watching the master znode so will be notified if another master dies.
203-
* @param checkInterval the interval to check if the master is stopped
204-
* @param startupStatus the monitor status to track the progress
204+
* @param checkInterval the interval to check if the master is stopped
205+
* @param startupTaskGroup the task group for master startup to track the progress
205206
* @return True if no issue becoming active master else false if another master was running or if
206207
* some other problem (zookeeper, stop flag has been set on this Master)
207208
*/
208-
boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupStatus) {
209+
boolean blockUntilBecomingActiveMaster(int checkInterval, TaskGroup startupTaskGroup) {
210+
MonitoredTask blockUntilActive =
211+
startupTaskGroup.addTask("Blocking until becoming active master");
209212
String backupZNode = ZNodePaths
210213
.joinZNode(this.watcher.getZNodePaths().backupMasterAddressesZNode, this.sn.toString());
211214
while (!(master.isAborted() || master.isStopped())) {
212-
startupStatus.setStatus("Trying to register in ZK as active master");
215+
blockUntilActive.setStatus("Trying to register in ZK as active master");
213216
// Try to become the active master, watch if there is another master.
214217
// Write out our ServerName as versioned bytes.
215218
try {
@@ -228,7 +231,7 @@ boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupS
228231
ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());
229232

230233
// We are the master, return
231-
startupStatus.setStatus("Successfully registered as active master.");
234+
blockUntilActive.setStatus("Successfully registered as active master.");
232235
this.clusterHasActiveMaster.set(true);
233236
activeMasterServerName = sn;
234237
LOG.info("Registered as active master=" + this.sn);
@@ -273,7 +276,7 @@ boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupS
273276
}
274277
}
275278
LOG.info(msg);
276-
startupStatus.setStatus(msg);
279+
blockUntilActive.setStatus(msg);
277280
} catch (KeeperException ke) {
278281
master.abort("Received an unexpected KeeperException, aborting", ke);
279282
return false;

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

Lines changed: 44 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -154,6 +154,7 @@
154154
import org.apache.hadoop.hbase.mob.MobConstants;
155155
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
156156
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
157+
import org.apache.hadoop.hbase.monitoring.TaskGroup;
157158
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
158159
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
159160
import org.apache.hadoop.hbase.procedure.flush.MasterFlushTableProcedureManager;
@@ -409,6 +410,11 @@ public class HMaster extends HRegionServer implements MasterServices {
409410
// Cached clusterId on stand by masters to serve clusterID requests from clients.
410411
private final CachedClusterId cachedClusterId;
411412

413+
public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move";
414+
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true;
415+
416+
private TaskGroup startupTaskGroup;
417+
412418
/**
413419
* Initializes the HMaster. The steps are as follows:
414420
* <p>
@@ -417,9 +423,8 @@ public class HMaster extends HRegionServer implements MasterServices {
417423
* <li>Start the ActiveMasterManager.
418424
* </ol>
419425
* <p>
420-
* Remaining steps of initialization occur in
421-
* {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
422-
* one.
426+
* Remaining steps of initialization occur in {@link #finishActiveMasterInitialization()} after
427+
* the master becomes the active one.
423428
*/
424429
public HMaster(final Configuration conf) throws IOException {
425430
super(conf);
@@ -792,12 +797,13 @@ protected AssignmentManager createAssignmentManager(MasterServices master) {
792797
* Notice that now we will not schedule a special procedure to make meta online(unless the first
793798
* time where meta has not been created yet), we will rely on SCP to bring meta online.
794799
*/
795-
private void finishActiveMasterInitialization(MonitoredTask status)
800+
801+
private void finishActiveMasterInitialization()
796802
throws IOException, InterruptedException, KeeperException, ReplicationException {
797803
/*
798804
* We are active master now... go initialize components we need to run.
799805
*/
800-
status.setStatus("Initializing Master file system");
806+
startupTaskGroup.addTask("Initializing Master file system");
801807

802808
this.masterActiveTime = System.currentTimeMillis();
803809
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -810,15 +816,15 @@ private void finishActiveMasterInitialization(MonitoredTask status)
810816

811817
// warm-up HTDs cache on master initialization
812818
if (preLoadTableDescriptors) {
813-
status.setStatus("Pre-loading table descriptors");
819+
startupTaskGroup.addTask("Pre-loading table descriptors");
814820
this.tableDescriptors.getAll();
815821
}
816822

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

@@ -837,7 +843,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
837843
}
838844
}
839845

840-
status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
846+
startupTaskGroup.addTask("Initialize ServerManager and schedule SCP for crash servers");
841847
this.serverManager = createServerManager(this);
842848
if (
843849
!conf.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK, DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)
@@ -881,8 +887,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
881887
? new MirroringTableStateManager(this)
882888
: new TableStateManager(this);
883889

884-
status.setStatus("Initializing ZK system trackers");
890+
startupTaskGroup.addTask("Initializing ZK system trackers");
885891
initializeZKBasedSystemTrackers();
892+
886893
// Set ourselves as active Master now our claim has succeeded up in zk.
887894
this.activeMaster = true;
888895

@@ -894,19 +901,19 @@ private void finishActiveMasterInitialization(MonitoredTask status)
894901

895902
// This is for backwards compatibility
896903
// See HBASE-11393
897-
status.setStatus("Update TableCFs node in ZNode");
904+
startupTaskGroup.addTask("Update TableCFs node in ZNode");
898905
ReplicationPeerConfigUpgrader tableCFsUpdater =
899906
new ReplicationPeerConfigUpgrader(zooKeeper, conf);
900907
tableCFsUpdater.copyTableCFs();
901908

902909
if (!maintenanceMode) {
903-
status.setStatus("Initializing master coprocessors");
910+
startupTaskGroup.addTask("Initializing master coprocessors");
904911
setQuotasObserver(conf);
905912
initializeCoprocessorHost(conf);
906913
}
907914

908915
// Checking if meta needs initializing.
909-
status.setStatus("Initializing meta table if this is a new deploy");
916+
startupTaskGroup.addTask("Initializing meta table if this is a new deploy");
910917
InitMetaProcedure initMetaProc = null;
911918
// Print out state of hbase:meta on startup; helps debugging.
912919
if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) {
@@ -929,7 +936,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
929936
this.balancer.initialize();
930937

931938
// start up all service threads.
932-
status.setStatus("Initializing master service threads");
939+
startupTaskGroup.addTask("Initializing master service threads");
933940
startServiceThreads();
934941
// wait meta to be initialized after we start procedure executor
935942
if (initMetaProc != null) {
@@ -942,16 +949,16 @@ private void finishActiveMasterInitialization(MonitoredTask status)
942949
// With this as part of master initialization, it precludes our being able to start a single
943950
// server that is both Master and RegionServer. Needs more thought. TODO.
944951
String statusStr = "Wait for region servers to report in";
945-
status.setStatus(statusStr);
946-
LOG.info(Objects.toString(status));
947-
waitForRegionServers(status);
952+
MonitoredTask waitRegionServer = startupTaskGroup.addTask(statusStr);
953+
LOG.info(Objects.toString(waitRegionServer));
954+
waitForRegionServers(waitRegionServer);
948955

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

954-
status.setStatus("Starting assignment manager");
961+
startupTaskGroup.addTask("Starting assignment manager");
955962
// FIRST HBASE:META READ!!!!
956963
// The below cannot make progress w/o hbase:meta being online.
957964
// This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
@@ -1028,7 +1035,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10281035
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
10291036

10301037
// Start balancer and meta catalog janitor after meta and regions have been assigned.
1031-
status.setStatus("Starting balancer and catalog janitor");
1038+
startupTaskGroup.addTask("Starting balancer and catalog janitor");
10321039
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
10331040
getChoreService().scheduleChore(clusterStatusChore);
10341041
this.balancerChore = new BalancerChore(this);
@@ -1050,7 +1057,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10501057
if (!waitForNamespaceOnline()) {
10511058
return;
10521059
}
1053-
status.setStatus("Starting cluster schema service");
1060+
startupTaskGroup.addTask("Starting cluster schema service");
10541061
try {
10551062
initClusterSchemaService();
10561063
} catch (IllegalStateException e) {
@@ -1073,7 +1080,6 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10731080
}
10741081
}
10751082

1076-
status.markComplete("Initialization successful");
10771083
LOG.info(String.format("Master has completed initialization %.3fsec",
10781084
(System.currentTimeMillis() - masterActiveTime) / 1000.0f));
10791085
this.masterFinishedInitializationTime = System.currentTimeMillis();
@@ -1085,6 +1091,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10851091
configurationManager.registerObserver(this.regionsRecoveryConfigManager);
10861092
// Set master as 'initialized'.
10871093
setInitialized(true);
1094+
startupTaskGroup.markComplete("Initialization successful");
1095+
MonitoredTask status =
1096+
TaskMonitor.get().createStatus("Progress after master initialized", false, true);
10881097

10891098
if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
10901099
// create missing CFs in meta table after master is set to 'initialized'.
@@ -1166,6 +1175,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
11661175
LOG.debug("Balancer post startup initialization complete, took "
11671176
+ ((System.currentTimeMillis() - start) / 1000) + " seconds");
11681177
}
1178+
status.markComplete("Progress after master initialized complete");
11691179
}
11701180

11711181
private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
@@ -2171,14 +2181,19 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
21712181
Threads.sleep(timeout);
21722182
}
21732183
}
2174-
MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
2175-
status.setDescription("Master startup");
2184+
2185+
// Here for the master startup process, we use TaskGroup to monitor the whole progress.
2186+
// The UI is similar to how Hadoop designed the startup page for the NameNode.
2187+
// See HBASE-21521 for more details.
2188+
// We do not cleanup the startupTaskGroup, let the startup progress information
2189+
// be permanent in the MEM.
2190+
startupTaskGroup = TaskMonitor.createTaskGroup(true, "Master startup");
21762191
try {
2177-
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
2178-
finishActiveMasterInitialization(status);
2192+
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
2193+
finishActiveMasterInitialization();
21792194
}
21802195
} catch (Throwable t) {
2181-
status.setStatus("Failed to become active: " + t.getMessage());
2196+
startupTaskGroup.abort("Failed to become active master due to:" + t.getMessage());
21822197
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
21832198
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
21842199
if (
@@ -2192,8 +2207,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
21922207
} else {
21932208
abort("Unhandled exception. Starting shutdown.", t);
21942209
}
2195-
} finally {
2196-
status.cleanup();
21972210
}
21982211
}
21992212

@@ -2756,6 +2769,10 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
27562769
return rsFatals;
27572770
}
27582771

2772+
public TaskGroup getStartupProgress() {
2773+
return startupTaskGroup;
2774+
}
2775+
27592776
/**
27602777
* Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
27612778
*/

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
@@ -129,8 +129,8 @@ public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices ma
129129
// prepare the verify
130130
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, workingDirFs);
131131
// update the running tasks
132-
this.status = TaskMonitor.get()
133-
.createStatus("Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, true);
132+
this.status = TaskMonitor.get().createStatus(
133+
"Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, false, true);
134134
this.snapshotManifest =
135135
SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor, status);
136136
}

0 commit comments

Comments
 (0)