Skip to content

Commit 297293c

Browse files
committed
HBASE-21521 Expose master startup status via web UI
1 parent efc7575 commit 297293c

File tree

8 files changed

+311
-39
lines changed

8 files changed

+311
-39
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: 40 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -181,6 +181,7 @@
181181
import org.apache.hadoop.hbase.mob.MobFileCompactionChore;
182182
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
183183
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
184+
import org.apache.hadoop.hbase.monitoring.TaskGroup;
184185
import org.apache.hadoop.hbase.monitoring.TaskMonitor;
185186
import org.apache.hadoop.hbase.namequeues.NamedQueueRecorder;
186187
import org.apache.hadoop.hbase.procedure.MasterProcedureManagerHost;
@@ -462,6 +463,8 @@ public class HMaster extends HBaseServerBase<MasterRpcServices> implements Maste
462463
public static final String WARMUP_BEFORE_MOVE = "hbase.master.warmup.before.move";
463464
private static final boolean DEFAULT_WARMUP_BEFORE_MOVE = true;
464465

466+
private TaskGroup startupTaskGroup;
467+
465468
/**
466469
* Initializes the HMaster. The steps are as follows:
467470
* <p>
@@ -908,12 +911,12 @@ private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperEx
908911
* Notice that now we will not schedule a special procedure to make meta online(unless the first
909912
* time where meta has not been created yet), we will rely on SCP to bring meta online.
910913
*/
911-
private void finishActiveMasterInitialization(MonitoredTask status)
914+
private void finishActiveMasterInitialization(TaskGroup startupTaskGroup)
912915
throws IOException, InterruptedException, KeeperException, ReplicationException {
913916
/*
914917
* We are active master now... go initialize components we need to run.
915918
*/
916-
status.setStatus("Initializing Master file system");
919+
startupTaskGroup.addTask("Initializing Master file system");
917920

918921
this.masterActiveTime = EnvironmentEdgeManager.currentTime();
919922
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -926,15 +929,15 @@ private void finishActiveMasterInitialization(MonitoredTask status)
926929

927930
// warm-up HTDs cache on master initialization
928931
if (preLoadTableDescriptors) {
929-
status.setStatus("Pre-loading table descriptors");
932+
startupTaskGroup.addTask("Pre-loading table descriptors");
930933
this.tableDescriptors.getAll();
931934
}
932935

933936
// Publish cluster ID; set it in Master too. The superclass RegionServer does this later but
934937
// only after it has checked in with the Master. At least a few tests ask Master for clusterId
935938
// before it has called its run method and before RegionServer has done the reportForDuty.
936939
ClusterId clusterId = fileSystemManager.getClusterId();
937-
status.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
940+
startupTaskGroup.addTask("Publishing Cluster ID " + clusterId + " in ZooKeeper");
938941
ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
939942
this.clusterId = clusterId.toString();
940943

@@ -953,7 +956,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
953956
}
954957
}
955958

956-
status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
959+
startupTaskGroup.addTask("Initialize ServerManager and schedule SCP for crash servers");
957960
// The below two managers must be created before loading procedures, as they will be used during
958961
// loading.
959962
// initialize master local region
@@ -1000,9 +1003,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10001003
// This manager must be accessed AFTER hbase:meta is confirmed on line..
10011004
this.tableStateManager = new TableStateManager(this);
10021005

1003-
status.setStatus("Initializing ZK system trackers");
1006+
startupTaskGroup.addTask("Initializing ZK system trackers");
10041007
initializeZKBasedSystemTrackers();
1005-
status.setStatus("Loading last flushed sequence id of regions");
1008+
startupTaskGroup.addTask("Loading last flushed sequence id of regions");
10061009
try {
10071010
this.serverManager.loadLastFlushedSequenceIds();
10081011
} catch (IOException e) {
@@ -1018,7 +1021,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10181021
zombieDetector.start();
10191022

10201023
if (!maintenanceMode) {
1021-
status.setStatus("Initializing master coprocessors");
1024+
startupTaskGroup.addTask("Initializing master coprocessors");
10221025
setQuotasObserver(conf);
10231026
initializeCoprocessorHost(conf);
10241027
} else {
@@ -1029,7 +1032,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10291032
}
10301033

10311034
// Checking if meta needs initializing.
1032-
status.setStatus("Initializing meta table if this is a new deploy");
1035+
startupTaskGroup.addTask("Initializing meta table if this is a new deploy");
10331036
InitMetaProcedure initMetaProc = null;
10341037
// Print out state of hbase:meta on startup; helps debugging.
10351038
if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) {
@@ -1049,7 +1052,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10491052
this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
10501053

10511054
// start up all service threads.
1052-
status.setStatus("Initializing master service threads");
1055+
startupTaskGroup.addTask("Initializing master service threads");
10531056
startServiceThreads();
10541057
// wait meta to be initialized after we start procedure executor
10551058
if (initMetaProc != null) {
@@ -1062,16 +1065,16 @@ private void finishActiveMasterInitialization(MonitoredTask status)
10621065
// With this as part of master initialization, it precludes our being able to start a single
10631066
// server that is both Master and RegionServer. Needs more thought. TODO.
10641067
String statusStr = "Wait for region servers to report in";
1065-
status.setStatus(statusStr);
1066-
LOG.info(Objects.toString(status));
1067-
waitForRegionServers(status);
1068+
MonitoredTask waitRegionServer = startupTaskGroup.addTask(statusStr);
1069+
LOG.info(Objects.toString(waitRegionServer));
1070+
waitForRegionServers(waitRegionServer);
10681071

10691072
// Check if master is shutting down because issue initializing regionservers or balancer.
10701073
if (isStopped()) {
10711074
return;
10721075
}
10731076

1074-
status.setStatus("Starting assignment manager");
1077+
startupTaskGroup.addTask("Starting assignment manager");
10751078
// FIRST HBASE:META READ!!!!
10761079
// The below cannot make progress w/o hbase:meta being online.
10771080
// This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
@@ -1136,7 +1139,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
11361139
this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
11371140

11381141
// Start balancer and meta catalog janitor after meta and regions have been assigned.
1139-
status.setStatus("Starting balancer and catalog janitor");
1142+
startupTaskGroup.addTask("Starting balancer and catalog janitor");
11401143
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
11411144
getChoreService().scheduleChore(clusterStatusChore);
11421145
this.balancerChore = new BalancerChore(this);
@@ -1156,7 +1159,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
11561159
if (!waitForNamespaceOnline()) {
11571160
return;
11581161
}
1159-
status.setStatus("Starting cluster schema service");
1162+
startupTaskGroup.addTask("Starting cluster schema service");
11601163
try {
11611164
initClusterSchemaService();
11621165
} catch (IllegalStateException e) {
@@ -1179,7 +1182,6 @@ private void finishActiveMasterInitialization(MonitoredTask status)
11791182
}
11801183
}
11811184

1182-
status.markComplete("Initialization successful");
11831185
LOG.info(String.format("Master has completed initialization %.3fsec",
11841186
(EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));
11851187
this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();
@@ -1198,6 +1200,8 @@ private void finishActiveMasterInitialization(MonitoredTask status)
11981200
}
11991201
// Set master as 'initialized'.
12001202
setInitialized(true);
1203+
startupTaskGroup.markComplete("Initialization successful");
1204+
MonitoredTask afterInitialized = startupTaskGroup.addTask("Progress after master initialized");
12011205

12021206
if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
12031207
// create missing CFs in meta table after master is set to 'initialized'.
@@ -1228,7 +1232,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
12281232
}
12291233

12301234
assignmentManager.checkIfShouldMoveSystemRegionAsync();
1231-
status.setStatus("Starting quota manager");
1235+
afterInitialized.setStatus("Starting quota manager");
12321236
initQuotaManager();
12331237
if (QuotaUtil.isQuotaEnabled(conf)) {
12341238
// Create the quota snapshot notifier
@@ -1251,13 +1255,13 @@ private void finishActiveMasterInitialization(MonitoredTask status)
12511255
this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
12521256

12531257
// Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
1254-
status.setStatus("Checking ZNode ACLs");
1258+
afterInitialized.setStatus("Checking ZNode ACLs");
12551259
zooKeeper.checkAndSetZNodeAcls();
12561260

1257-
status.setStatus("Initializing MOB Cleaner");
1261+
afterInitialized.setStatus("Initializing MOB Cleaner");
12581262
initMobCleaner();
12591263

1260-
status.setStatus("Calling postStartMaster coprocessors");
1264+
afterInitialized.setStatus("Calling postStartMaster coprocessors");
12611265
if (this.cpHost != null) {
12621266
// don't let cp initialization errors kill the master
12631267
try {
@@ -1282,6 +1286,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
12821286

12831287
this.rollingUpgradeChore = new RollingUpgradeChore(this);
12841288
getChoreService().scheduleChore(rollingUpgradeChore);
1289+
afterInitialized.markComplete("Progress after master initialized complete");
12851290
}
12861291

12871292
private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
@@ -2401,14 +2406,16 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
24012406
Threads.sleep(timeout);
24022407
}
24032408
}
2404-
MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
2405-
status.setDescription("Master startup");
2409+
boolean ignoreClearStartupStatus =
2410+
conf.getBoolean("hbase.master.ignore.clear.startup.status", true);
2411+
startupTaskGroup = TaskGroup.createTaskGroup(ignoreClearStartupStatus);
2412+
startupTaskGroup.setDescription("Master startup");
24062413
try {
2407-
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
2408-
finishActiveMasterInitialization(status);
2414+
if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
2415+
finishActiveMasterInitialization(startupTaskGroup);
24092416
}
24102417
} catch (Throwable t) {
2411-
status.setStatus("Failed to become active: " + t.getMessage());
2418+
startupTaskGroup.abort("Failed to become active master");
24122419
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
24132420
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
24142421
if (
@@ -2423,7 +2430,9 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
24232430
abort("Unhandled exception. Starting shutdown.", t);
24242431
}
24252432
} finally {
2426-
status.cleanup();
2433+
if (!ignoreClearStartupStatus) {
2434+
startupTaskGroup.cleanup();
2435+
}
24272436
}
24282437
}
24292438

@@ -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
*/
Lines changed: 95 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,95 @@
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+
@InterfaceAudience.Private
28+
public class TaskGroup extends MonitoredTaskImpl {
29+
private static final Logger LOG = LoggerFactory.getLogger(TaskGroup.class);
30+
31+
private final ConcurrentLinkedDeque<MonitoredTask> tasks = new ConcurrentLinkedDeque<>();
32+
private final boolean ignoreClearStatus;
33+
34+
public TaskGroup(boolean ignoreClearStatus) {
35+
super(false);
36+
this.ignoreClearStatus = ignoreClearStatus;
37+
}
38+
39+
public TaskGroup() {
40+
this(false);
41+
}
42+
43+
public static TaskGroup createTaskGroup(boolean ignoreClearStatus) {
44+
return new TaskGroup(ignoreClearStatus);
45+
}
46+
47+
public synchronized MonitoredTask addTask(String description) {
48+
return addTask(description, true);
49+
}
50+
51+
public synchronized MonitoredTask addTask(String description, boolean withCompleteLast) {
52+
if (withCompleteLast) {
53+
MonitoredTask previousTask = this.tasks.peekLast();
54+
if (
55+
previousTask != null && previousTask.getState() != State.COMPLETE
56+
&& previousTask.getState() != State.ABORTED
57+
) {
58+
previousTask.markComplete("Completed");
59+
}
60+
}
61+
MonitoredTask task = TaskMonitor.get().createStatus(description, ignoreClearStatus, true);
62+
this.setStatus(description);
63+
this.tasks.addLast(task);
64+
return task;
65+
}
66+
67+
public synchronized Collection<MonitoredTask> getTasks() {
68+
return Collections.unmodifiableCollection(this.tasks);
69+
}
70+
71+
@Override
72+
public synchronized void abort(String msg) {
73+
setStatus(msg);
74+
setState(State.ABORTED);
75+
for (MonitoredTask task : tasks) {
76+
if (task.getState() != State.COMPLETE && task.getState() != State.ABORTED) {
77+
task.abort(msg);
78+
}
79+
}
80+
}
81+
82+
@Override
83+
public synchronized void markComplete(String msg) {
84+
setState(State.COMPLETE);
85+
setStatus(msg);
86+
if (tasks.getLast() != null) {
87+
tasks.getLast().markComplete(msg);
88+
}
89+
}
90+
91+
@Override
92+
public synchronized void cleanup() {
93+
this.tasks.clear();
94+
}
95+
}

0 commit comments

Comments
 (0)