diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon index 8cebaaa2d1f5..c71b66c462e8 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/master/MasterStatusTmpl.jamon @@ -167,6 +167,7 @@ AssignmentManager assignmentManager = master.getAssignmentManager(); <%if HBaseConfiguration.isShowConfInServlet()%>
@@ -417,9 +423,8 @@ public class HMaster extends HRegionServer implements MasterServices { *
- * Remaining steps of initialization occur in
- * {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the active
- * one.
+ * Remaining steps of initialization occur in {@link #finishActiveMasterInitialization()} after
+ * the master becomes the active one.
*/
public HMaster(final Configuration conf) throws IOException {
super(conf);
@@ -792,12 +797,13 @@ protected AssignmentManager createAssignmentManager(MasterServices master) {
* Notice that now we will not schedule a special procedure to make meta online(unless the first
* time where meta has not been created yet), we will rely on SCP to bring meta online.
*/
- private void finishActiveMasterInitialization(MonitoredTask status)
+
+ private void finishActiveMasterInitialization()
throws IOException, InterruptedException, KeeperException, ReplicationException {
/*
* We are active master now... go initialize components we need to run.
*/
- status.setStatus("Initializing Master file system");
+ startupTaskGroup.addTask("Initializing Master file system");
this.masterActiveTime = System.currentTimeMillis();
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -810,7 +816,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
// warm-up HTDs cache on master initialization
if (preLoadTableDescriptors) {
- status.setStatus("Pre-loading table descriptors");
+ startupTaskGroup.addTask("Pre-loading table descriptors");
this.tableDescriptors.getAll();
}
@@ -818,7 +824,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
// only after it has checked in with the Master. At least a few tests ask Master for clusterId
// before it has called its run method and before RegionServer has done the reportForDuty.
ClusterId clusterId = fileSystemManager.getClusterId();
- status.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
+ startupTaskGroup.addTask("Publishing Cluster ID " + clusterId + " in ZooKeeper");
ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
this.clusterId = clusterId.toString();
@@ -837,7 +843,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
}
- status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
+ startupTaskGroup.addTask("Initialize ServerManager and schedule SCP for crash servers");
this.serverManager = createServerManager(this);
if (
!conf.getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK, DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)
@@ -881,8 +887,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
? new MirroringTableStateManager(this)
: new TableStateManager(this);
- status.setStatus("Initializing ZK system trackers");
+ startupTaskGroup.addTask("Initializing ZK system trackers");
initializeZKBasedSystemTrackers();
+
// Set ourselves as active Master now our claim has succeeded up in zk.
this.activeMaster = true;
@@ -894,19 +901,19 @@ private void finishActiveMasterInitialization(MonitoredTask status)
// This is for backwards compatibility
// See HBASE-11393
- status.setStatus("Update TableCFs node in ZNode");
+ startupTaskGroup.addTask("Update TableCFs node in ZNode");
ReplicationPeerConfigUpgrader tableCFsUpdater =
new ReplicationPeerConfigUpgrader(zooKeeper, conf);
tableCFsUpdater.copyTableCFs();
if (!maintenanceMode) {
- status.setStatus("Initializing master coprocessors");
+ startupTaskGroup.addTask("Initializing master coprocessors");
setQuotasObserver(conf);
initializeCoprocessorHost(conf);
}
// Checking if meta needs initializing.
- status.setStatus("Initializing meta table if this is a new deploy");
+ startupTaskGroup.addTask("Initializing meta table if this is a new deploy");
InitMetaProcedure initMetaProc = null;
// Print out state of hbase:meta on startup; helps debugging.
if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) {
@@ -929,7 +936,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
this.balancer.initialize();
// start up all service threads.
- status.setStatus("Initializing master service threads");
+ startupTaskGroup.addTask("Initializing master service threads");
startServiceThreads();
// wait meta to be initialized after we start procedure executor
if (initMetaProc != null) {
@@ -942,16 +949,16 @@ private void finishActiveMasterInitialization(MonitoredTask status)
// With this as part of master initialization, it precludes our being able to start a single
// server that is both Master and RegionServer. Needs more thought. TODO.
String statusStr = "Wait for region servers to report in";
- status.setStatus(statusStr);
- LOG.info(Objects.toString(status));
- waitForRegionServers(status);
+ MonitoredTask waitRegionServer = startupTaskGroup.addTask(statusStr);
+ LOG.info(Objects.toString(waitRegionServer));
+ waitForRegionServers(waitRegionServer);
// Check if master is shutting down because issue initializing regionservers or balancer.
if (isStopped()) {
return;
}
- status.setStatus("Starting assignment manager");
+ startupTaskGroup.addTask("Starting assignment manager");
// FIRST HBASE:META READ!!!!
// The below cannot make progress w/o hbase:meta being online.
// 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)
this.balancer.setClusterMetrics(getClusterMetricsWithoutCoprocessor());
// Start balancer and meta catalog janitor after meta and regions have been assigned.
- status.setStatus("Starting balancer and catalog janitor");
+ startupTaskGroup.addTask("Starting balancer and catalog janitor");
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
getChoreService().scheduleChore(clusterStatusChore);
this.balancerChore = new BalancerChore(this);
@@ -1050,7 +1057,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
if (!waitForNamespaceOnline()) {
return;
}
- status.setStatus("Starting cluster schema service");
+ startupTaskGroup.addTask("Starting cluster schema service");
try {
initClusterSchemaService();
} catch (IllegalStateException e) {
@@ -1073,7 +1080,6 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
}
- status.markComplete("Initialization successful");
LOG.info(String.format("Master has completed initialization %.3fsec",
(System.currentTimeMillis() - masterActiveTime) / 1000.0f));
this.masterFinishedInitializationTime = System.currentTimeMillis();
@@ -1085,6 +1091,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
configurationManager.registerObserver(this.regionsRecoveryConfigManager);
// Set master as 'initialized'.
setInitialized(true);
+ startupTaskGroup.markComplete("Initialization successful");
+ MonitoredTask status =
+ TaskMonitor.get().createStatus("Progress after master initialized", false, true);
if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
// create missing CFs in meta table after master is set to 'initialized'.
@@ -1166,6 +1175,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
LOG.debug("Balancer post startup initialization complete, took "
+ ((System.currentTimeMillis() - start) / 1000) + " seconds");
}
+ status.markComplete("Progress after master initialized complete");
}
private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
@@ -2171,14 +2181,19 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
Threads.sleep(timeout);
}
}
- MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
- status.setDescription("Master startup");
+
+ // Here for the master startup process, we use TaskGroup to monitor the whole progress.
+ // The UI is similar to how Hadoop designed the startup page for the NameNode.
+ // See HBASE-21521 for more details.
+ // We do not cleanup the startupTaskGroup, let the startup progress information
+ // be permanent in the MEM.
+ startupTaskGroup = TaskMonitor.createTaskGroup(true, "Master startup");
try {
- if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
- finishActiveMasterInitialization(status);
+ if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
+ finishActiveMasterInitialization();
}
} catch (Throwable t) {
- status.setStatus("Failed to become active: " + t.getMessage());
+ startupTaskGroup.abort("Failed to become active master due to:" + t.getMessage());
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
if (
@@ -2192,8 +2207,6 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
} else {
abort("Unhandled exception. Starting shutdown.", t);
}
- } finally {
- status.cleanup();
}
}
@@ -2756,6 +2769,10 @@ public MemoryBoundedLogMessageBuffer getRegionServerFatalLogBuffer() {
return rsFatals;
}
+ public TaskGroup getStartupProgress() {
+ return startupTaskGroup;
+ }
+
/**
* Shutdown the cluster. Master runs a coordinated stop of all RegionServers and then itself.
*/
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index acd8ce972a6e..5a4ce31add9c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -130,7 +130,7 @@ public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices ma
this.verifier = new MasterSnapshotVerifier(masterServices, snapshot, workingDirFs);
// update the running tasks
this.status = TaskMonitor.get()
- .createStatus("Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, true);
+ .createStatus("Taking " + snapshot.getType() + " snapshot on table: " + snapshotTable, false);
this.snapshotManifest =
SnapshotManifest.create(conf, rootFs, workingDir, snapshot, monitor, status);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskGroup.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskGroup.java
new file mode 100644
index 000000000000..e2a7fef8258f
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskGroup.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.monitoring;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The {@link TaskGroup} can be seen as a big {@link MonitoredTask}, which contains a list of sub
+ * monitored tasks. The monitored tasks in the group are still be managed by the
+ * {@link TaskMonitor}, but whether to clear/expire the monitored tasks in a task group is optional.
+ * Since the monitored task already has journals, which mark the phases in a task, we still also
+ * need a task group to monitor a big task/process because the journals in a task is serial but the
+ * tasks in the task group can be parallel, then we have more flexible ability to monitor the
+ * process. Grouping the tasks is not strictly necessary but it is cleaner for presentation to
+ * operators. We might want to display the tasks in a group in a list view where each task can be
+ * collapsed (probably by default) or expanded.
+ */
+@InterfaceAudience.Private
+public class TaskGroup extends MonitoredTaskImpl {
+ private static final Logger LOG = LoggerFactory.getLogger(TaskGroup.class);
+
+ /** Sub-tasks in the group */
+ private final ConcurrentLinkedDequeStartup Progress (
+ <% TaskGroup startupTaskGroup = master.getStartupProgress();
+ if(startupTaskGroup != null){ %>
+ <%= getStartupStatusString(startupTaskGroup) %>
+ <% } else { %>
+ <%= ""%>
+ <% } %>
+ )
+
+
+
+
+
+ <%
+ if(startupTaskGroup != null){
+ for (MonitoredTask task : startupTaskGroup.getTasks()) { %>
+ Task
+ Current State
+ Start Time
+ Last status Time
+ Elapsed Time(ms)
+ Journals
+
+
+
+ <% }
+ } %>
+
+ <%= task.getDescription() %>
+ <%= task.getState().name() %>
+ <%= new Date(task.getStartTime()) %>
+ <%= new Date(task.getStatusTime()) %>
+ <%= task.getStatusTime() - task.getStartTime() %>
+ <%= printLatestJournals(task, 30) %>
+