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 1bfa967d70f9..60e40e570e93 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()%>
@@ -908,12 +911,12 @@ private void tryMigrateMetaLocationsFromZooKeeper() throws IOException, KeeperEx
* 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(TaskGroup startupTaskGroup)
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 = EnvironmentEdgeManager.currentTime();
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -926,7 +929,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();
}
@@ -934,7 +937,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();
@@ -953,7 +956,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");
// The below two managers must be created before loading procedures, as they will be used during
// loading.
// initialize master local region
@@ -1000,9 +1003,9 @@ private void finishActiveMasterInitialization(MonitoredTask status)
// This manager must be accessed AFTER hbase:meta is confirmed on line..
this.tableStateManager = new TableStateManager(this);
- status.setStatus("Initializing ZK system trackers");
+ startupTaskGroup.addTask("Initializing ZK system trackers");
initializeZKBasedSystemTrackers();
- status.setStatus("Loading last flushed sequence id of regions");
+ startupTaskGroup.addTask("Loading last flushed sequence id of regions");
try {
this.serverManager.loadLastFlushedSequenceIds();
} catch (IOException e) {
@@ -1018,7 +1021,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
zombieDetector.start();
if (!maintenanceMode) {
- status.setStatus("Initializing master coprocessors");
+ startupTaskGroup.addTask("Initializing master coprocessors");
setQuotasObserver(conf);
initializeCoprocessorHost(conf);
} else {
@@ -1029,7 +1032,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
// 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)) {
@@ -1049,7 +1052,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
// 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) {
@@ -1062,16 +1065,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
@@ -1136,7 +1139,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
this.balancer.updateClusterMetrics(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);
@@ -1156,7 +1159,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) {
@@ -1179,7 +1182,6 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
}
- status.markComplete("Initialization successful");
LOG.info(String.format("Master has completed initialization %.3fsec",
(EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));
this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();
@@ -1198,6 +1200,8 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
// Set master as 'initialized'.
setInitialized(true);
+ startupTaskGroup.markComplete("Initialization successful");
+ MonitoredTask afterInitialized = startupTaskGroup.addTask("Progress after master initialized");
if (tableFamilyDesc == null && replBarrierFamilyDesc == null) {
// create missing CFs in meta table after master is set to 'initialized'.
@@ -1228,7 +1232,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
}
assignmentManager.checkIfShouldMoveSystemRegionAsync();
- status.setStatus("Starting quota manager");
+ afterInitialized.setStatus("Starting quota manager");
initQuotaManager();
if (QuotaUtil.isQuotaEnabled(conf)) {
// Create the quota snapshot notifier
@@ -1251,13 +1255,13 @@ private void finishActiveMasterInitialization(MonitoredTask status)
this.serverManager.clearDeadServersWithSameHostNameAndPortOfOnlineServer();
// Check and set the znode ACLs if needed in case we are overtaking a non-secure configuration
- status.setStatus("Checking ZNode ACLs");
+ afterInitialized.setStatus("Checking ZNode ACLs");
zooKeeper.checkAndSetZNodeAcls();
- status.setStatus("Initializing MOB Cleaner");
+ afterInitialized.setStatus("Initializing MOB Cleaner");
initMobCleaner();
- status.setStatus("Calling postStartMaster coprocessors");
+ afterInitialized.setStatus("Calling postStartMaster coprocessors");
if (this.cpHost != null) {
// don't let cp initialization errors kill the master
try {
@@ -1282,6 +1286,7 @@ private void finishActiveMasterInitialization(MonitoredTask status)
this.rollingUpgradeChore = new RollingUpgradeChore(this);
getChoreService().scheduleChore(rollingUpgradeChore);
+ afterInitialized.markComplete("Progress after master initialized complete");
}
private void createMissingCFsInMetaDuringUpgrade(TableDescriptor metaDescriptor)
@@ -2401,14 +2406,16 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
Threads.sleep(timeout);
}
}
- MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
- status.setDescription("Master startup");
+ boolean ignoreClearStartupStatus =
+ conf.getBoolean("hbase.master.ignore.clear.startup.status", true);
+ startupTaskGroup = TaskGroup.createTaskGroup(ignoreClearStartupStatus);
+ startupTaskGroup.setDescription("Master startup");
try {
- if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
- finishActiveMasterInitialization(status);
+ if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupTaskGroup)) {
+ finishActiveMasterInitialization(startupTaskGroup);
}
} catch (Throwable t) {
- status.setStatus("Failed to become active: " + t.getMessage());
+ startupTaskGroup.abort("Failed to become active master");
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
if (
@@ -2423,7 +2430,9 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
abort("Unhandled exception. Starting shutdown.", t);
}
} finally {
- status.cleanup();
+ if (!ignoreClearStartupStatus) {
+ startupTaskGroup.cleanup();
+ }
}
}
@@ -3099,6 +3108,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/monitoring/TaskGroup.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskGroup.java
new file mode 100644
index 000000000000..432cd76d39b4
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/TaskGroup.java
@@ -0,0 +1,95 @@
+/*
+ * 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;
+
+@InterfaceAudience.Private
+public class TaskGroup extends MonitoredTaskImpl {
+ private static final Logger LOG = LoggerFactory.getLogger(TaskGroup.class);
+
+ 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) %>
+