Skip to content

Commit 41fff2a

Browse files
committed
HBASE-21521 Expose master startup status via web UI
1 parent fa6b5d1 commit 41fff2a

File tree

10 files changed

+337
-44
lines changed

10 files changed

+337
-44
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: 39 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -186,6 +186,7 @@
186186
import org.apache.hadoop.hbase.mob.MobFileCompactionChore;
187187
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
188188
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
189+
import org.apache.hadoop.hbase.monitoring.TaskGroup;
189190
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
190191
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
191192
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
@@ -465,6 +466,8 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
465466
public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move";
466467
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true;
467468

469+
private TaskGroup startupTaskGroup;
470+
468471
/**
469472
* Initializes the HMaster. The steps are as follows:
470473
* <p>
@@ -473,9 +476,8 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
473476
* <li>Start the ActiveMasterManager.
474477
* </ol>
475478
* <p>
476-
* Remaining steps of initialization occur in
477-
* {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
478-
* one.
479+
* Remaining steps of initialization occur in {@link #finishActiveMasterInitialization(TaskGroup)}
480+
* after the master becomes the active one.
479481
*/
480482
public HMaster(final Configuration conf) throws IOException {
481483
super(conf, "Master");
@@ -908,12 +910,12 @@ private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperEx
908910
* Notice that now we will not schedule a special procedure to make meta online(unless the first
909911
* time where meta has not been created yet), we will rely on SCP to bring meta online.
910912
*/
911-
private void finishActiveMasterInitialization(MonitoredTask status) throws IOException,
913+
private void finishActiveMasterInitialization(TaskGroup startupTaskGro) throws IOException,
912914
InterruptedException, KeeperException, ReplicationException, DeserializationException {
913915
/*
914916
* We are active master now... go initialize components we need to run.
915917
*/
916-
status.setStatus("Initializing Master file system");
918+
startupTaskGroup.addTask("Initializing Master file system");
917919

918920
this.masterActiveTime = EnvironmentEdgeManager.currentTime();
919921
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -926,15 +928,15 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
926928

927929
// warm-up HTDs cache on master initialization
928930
if (preLoadTableDescriptors) {
929-
status.setStatus("Pre-loading table descriptors");
931+
startupTaskGroup.addTask("Pre-loading table descriptors");
930932
this.tableDescriptors.getAll();
931933
}
932934

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

@@ -953,7 +955,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
953955
}
954956
}
955957

956-
status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
958+
startupTaskGroup.addTask("Initialize ServerManager and schedule SCP for crash servers");
957959
// The below two managers must be created before loading procedures, as they will be used during
958960
// loading.
959961
// initialize master local region
@@ -1000,9 +1002,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
10001002
// This manager must be accessed AFTER hbase:meta is confirmed on line..
10011003
this.tableStateManager = new TableStateManager(this);
10021004

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

10201022
if (!maintenanceMode) {
1021-
status.setStatus("Initializing master coprocessors");
1023+
startupTaskGroup.addTask("Initializing master coprocessors");
10221024
setQuotasObserver(conf);
10231025
initializeCoprocessorHost(conf);
10241026
} else {
@@ -1029,7 +1031,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
10291031
}
10301032

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

10511053
// start up all service threads.
1052-
status.setStatus("Initializing master service threads");
1054+
startupTaskGroup.addTask("Initializing master service threads");
10531055
startServiceThreads();
10541056
// wait meta to be initialized after we start procedure executor
10551057
if (initMetaProc != null) {
@@ -1062,16 +1064,16 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
10621064
// With this as part of master initialization, it precludes our being able to start a single
10631065
// server that is both Master and RegionServer. Needs more thought. TODO.
10641066
String statusStr = "Wait for region servers to report in";
1065-
status.setStatus(statusStr);
1066-
LOG.info(Objects.toString(status));
1067-
waitForRegionServers(status);
1067+
MonitoredTask waitRegionServer = startupTaskGroup.addTask(statusStr);
1068+
LOG.info(Objects.toString(waitRegionServer));
1069+
waitForRegionServers(waitRegionServer);
10681070

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

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

11381140
// Start balancer and meta catalog janitor after meta and regions have been assigned.
1139-
status.setStatus("Starting balancer and catalog janitor");
1141+
startupTaskGroup.addTask("Starting balancer and catalog janitor");
11401142
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
11411143
getChoreService().scheduleChore(clusterStatusChore);
11421144
this.balancerChore = new BalancerChore(this);
@@ -1156,7 +1158,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
11561158
if (!waitForNamespaceOnline()) {
11571159
return;
11581160
}
1159-
status.setStatus("Starting cluster schema service");
1161+
startupTaskGroup.addTask("Starting cluster schema service");
11601162
try {
11611163
initClusterSchemaService();
11621164
} catch (IllegalStateException e) {
@@ -1179,7 +1181,6 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
11791181
}
11801182
}
11811183

1182-
status.markComplete("Initialization successful");
11831184
LOG.info(String.format("Master has completed initialization %.3fsec",
11841185
(EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));
11851186
this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();
@@ -1198,6 +1199,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
11981199
}
11991200
// Set master as 'initialized'.
12001201
setInitialized(true);
1202+
startupTaskGroup.markComplete("Initialization successful");
1203+
MonitoredTask status =
1204+
TaskMonitor.get().createStatus("Progress after master initialized", false, true);
12011205

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

12871291
this.rollingUpgradeChore = new RollingUpgradeChore(this);
12881292
getChoreService().scheduleChore(rollingUpgradeChore);
1293+
status.markComplete("Progress after master initialized complete");
12891294
}
12901295

12911296
private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
@@ -2399,14 +2404,19 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
23992404
Threads.sleep(timeout);
24002405
}
24012406
}
2402-
MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
2403-
status.setDescription("Master startup");
2407+
2408+
// Here for the master startup process, we use TaskGroup to monitor the whole progress.
2409+
// The UI is similar to how Hadoop designed the startup page for the NameNode.
2410+
// See HBASE-21521 for more details.
2411+
// We do not cleanup the startupTaskGroup, let the startup progress information
2412+
// be permanent in the MEM.
2413+
startupTaskGroup = TaskGroup.createTaskGroup(false, "Master startup");
24042414
try {
2405-
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
2406-
finishActiveMasterInitialization(status);
2415+
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
2416+
finishActiveMasterInitialization(startupTaskGroup);
24072417
}
24082418
} catch (Throwable t) {
2409-
status.setStatus("Failed to become active: " + t.getMessage());
2419+
startupTaskGroup.abort("Failed to become active master due to:" + t.getMessage());
24102420
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
24112421
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
24122422
if (
@@ -2420,8 +2430,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
24202430
} else {
24212431
abort("Unhandled exception. Starting shutdown.", t);
24222432
}
2423-
} finally {
2424-
status.cleanup();
24252433
}
24262434
}
24272435

@@ -3097,6 +3105,10 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
30973105
return rsFatals;
30983106
}
30993107

3108+
public TaskGroup getStartupProgress() {
3109+
return startupTaskGroup;
3110+
}
3111+
31003112
/**
31013113
* Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
31023114
*/
Lines changed: 116 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,116 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hbase.monitoring;
19+
20+
import java.util.Collection;
21+
import java.util.Collections;
22+
import java.util.concurrent.ConcurrentLinkedDeque;
23+
import org.apache.yetus.audience.InterfaceAudience;
24+
import org.slf4j.Logger;
25+
import org.slf4j.LoggerFactory;
26+
27+
/**
28+
* The {@link TaskGroup} can be seen as a big {@link MonitoredTask}, which contains a list of sub
29+
* monitored tasks. The monitored tasks in the group are still be managed by the
30+
* {@link TaskMonitor}, but whether to clear/expire the monitored tasks in a task group is optional.
31+
* Since the monitored task already has journals, which mark the phases in a task, we still also
32+
* need a task group to monitor a big task/process because the journals in a task is serial but the
33+
* tasks in the task group can be parallel, then we have more flexible ability to monitor the
34+
* process. Grouping the tasks is not strictly necessary but it is cleaner for presentation to
35+
* operators. We might want to display the tasks in a group in a list view where each task can be
36+
* collapsed (probably by default) or expanded.
37+
*/
38+
@InterfaceAudience.Private
39+
public class TaskGroup extends MonitoredTaskImpl {
40+
private static final Logger LOG = LoggerFactory.getLogger(TaskGroup.class);
41+
42+
private final ConcurrentLinkedDeque<MonitoredTask> tasks = new ConcurrentLinkedDeque<>();
43+
44+
/** Whether to be tracked(e.g. show/clear/expire) in the singleton {@link TaskMonitor} */
45+
private final boolean trackSubTasksInTaskMonitor;
46+
private final MonitoredTask delegate;
47+
48+
public TaskGroup(boolean trackSubTasksInTaskMonitor, String description) {
49+
super(false, description);
50+
this.trackSubTasksInTaskMonitor = trackSubTasksInTaskMonitor;
51+
this.delegate = TaskMonitor.get().createStatus(description, true, true);
52+
}
53+
54+
public static TaskGroup createTaskGroup(boolean trackSubTasksInTaskMonitor, String description) {
55+
return new TaskGroup(trackSubTasksInTaskMonitor, description);
56+
}
57+
58+
public synchronized MonitoredTask addTask(String description) {
59+
return addTask(description, true);
60+
}
61+
62+
/**
63+
* add a new task to the group, and before that might complete the last task in the group
64+
* @param description the description of the new task
65+
* @param withCompleteLast whether to complete the last task in the group
66+
* @return the added new task
67+
*/
68+
public synchronized MonitoredTask addTask(String description, boolean withCompleteLast) {
69+
if (withCompleteLast) {
70+
MonitoredTask previousTask = this.tasks.peekLast();
71+
if (
72+
previousTask != null && previousTask.getState() != State.COMPLETE
73+
&& previousTask.getState() != State.ABORTED
74+
) {
75+
previousTask.markComplete("Completed");
76+
}
77+
}
78+
MonitoredTask task =
79+
TaskMonitor.get().createStatus(description, trackSubTasksInTaskMonitor, true);
80+
this.setStatus(description);
81+
this.tasks.addLast(task);
82+
delegate.setStatus(description);
83+
return task;
84+
}
85+
86+
public synchronized Collection<MonitoredTask> getTasks() {
87+
return Collections.unmodifiableCollection(this.tasks);
88+
}
89+
90+
@Override
91+
public synchronized void abort(String msg) {
92+
setStatus(msg);
93+
setState(State.ABORTED);
94+
for (MonitoredTask task : tasks) {
95+
if (task.getState() != State.COMPLETE && task.getState() != State.ABORTED) {
96+
task.abort(msg);
97+
}
98+
}
99+
delegate.abort(msg);
100+
}
101+
102+
@Override
103+
public synchronized void markComplete(String msg) {
104+
setState(State.COMPLETE);
105+
setStatus(msg);
106+
if (tasks.getLast() != null) {
107+
tasks.getLast().markComplete(msg);
108+
}
109+
delegate.markComplete(msg);
110+
}
111+
112+
@Override
113+
public synchronized void cleanup() {
114+
this.tasks.clear();
115+
}
116+
}

0 commit comments

Comments
 (0)