implements Maste
*
*
* Remaining steps of initialization occur in
- * {@link #finishActiveMasterInitialization(MonitoredTask)} after the master becomes the
+ * {@link #finishActiveMasterInitialization()} after the master becomes the
* active one.
*/
public HMaster(final Configuration conf) throws IOException {
@@ -855,12 +858,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) throws IOException,
+ 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");
+ startupStatusTask.setStatus("Initializing Master file system");
this.masterActiveTime = EnvironmentEdgeManager.currentTime();
// TODO: Do this using Dependency Injection, using PicoContainer, Guice or Spring.
@@ -873,7 +876,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// warm-up HTDs cache on master initialization
if (preLoadTableDescriptors) {
- status.setStatus("Pre-loading table descriptors");
+ startupStatusTask.setStatus("Pre-loading table descriptors");
this.tableDescriptors.getAll();
}
@@ -881,7 +884,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// 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");
+ startupStatusTask.setStatus("Publishing Cluster ID " + clusterId + " in ZooKeeper");
ZKClusterId.setClusterId(this.zooKeeper, fileSystemManager.getClusterId());
this.clusterId = clusterId.toString();
@@ -900,7 +903,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
}
}
- status.setStatus("Initialize ServerManager and schedule SCP for crash servers");
+ startupStatusTask.setStatus("Initialize ServerManager");
// The below two managers must be created before loading procedures, as they will be used during
// loading.
this.serverManager = createServerManager(this);
@@ -913,6 +916,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// initialize master local region
masterRegion = MasterRegionFactory.create(this);
+ startupStatusTask.setStatus("Migrate Meta Locations From Zookeeper");
tryMigrateMetaLocationsFromZooKeeper();
createProcedureExecutor();
@@ -921,6 +925,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
.collect(Collectors.groupingBy(p -> p.getClass()));
// Create Assignment Manager
+ startupStatusTask.setStatus("Create Assignment Manager");
this.assignmentManager = createAssignmentManager(this, masterRegion);
this.assignmentManager.start();
// TODO: TRSP can perform as the sub procedure for other procedures, so even if it is marked as
@@ -937,6 +942,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// filesystem that COULD BE 'alive' (we'll schedule SCPs for each and let SCP figure it out).
// We also pass dirs that are already 'splitting'... so we can do some checks down in tracker.
// TODO: Generate the splitting and live Set in one pass instead of two as we currently do.
+ startupStatusTask.setStatus("Schedule SCP for crash servers");
this.regionServerTracker.upgrade(
procsByType.getOrDefault(ServerCrashProcedure.class, Collections.emptyList()).stream()
.map(p -> (ServerCrashProcedure) p).map(p -> p.getServerName()).collect(Collectors.toSet()),
@@ -944,9 +950,9 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// This manager must be accessed AFTER hbase:meta is confirmed on line..
this.tableStateManager = new TableStateManager(this);
- status.setStatus("Initializing ZK system trackers");
+ startupStatusTask.setStatus("Initializing ZK system trackers");
initializeZKBasedSystemTrackers();
- status.setStatus("Loading last flushed sequence id of regions");
+ startupStatusTask.setStatus("Loading last flushed sequence id of regions");
try {
this.serverManager.loadLastFlushedSequenceIds();
} catch (IOException e) {
@@ -969,7 +975,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
updateConfigurationForQuotasObserver(conf);
}
// initialize master side coprocessors before we start handling requests
- status.setStatus("Initializing master coprocessors");
+ startupStatusTask.setStatus("Initializing master coprocessors");
this.cpHost = new MasterCoprocessorHost(this, this.conf);
} else {
// start an in process region server for carrying system regions
@@ -979,7 +985,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
}
// Checking if meta needs initializing.
- status.setStatus("Initializing meta table if this is a new deploy");
+ startupStatusTask.setStatus("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)) {
@@ -999,7 +1005,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
// start up all service threads.
- status.setStatus("Initializing master service threads");
+ startupStatusTask.setStatus("Initializing master service threads");
startServiceThreads();
// wait meta to be initialized after we start procedure executor
if (initMetaProc != null) {
@@ -1012,16 +1018,15 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
// 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);
+ LOG.info(Objects.toString(startupStatusTask));
+ waitForRegionServers(startupStatusTask);
// Check if master is shutting down because issue initializing regionservers or balancer.
if (isStopped()) {
return;
}
- status.setStatus("Starting assignment manager");
+ startupStatusTask.setStatus("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
@@ -1087,7 +1092,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
this.balancer.updateClusterMetrics(getClusterMetricsWithoutCoprocessor());
// Start balancer and meta catalog janitor after meta and regions have been assigned.
- status.setStatus("Starting balancer and catalog janitor");
+ startupStatusTask.setStatus("Starting balancer and catalog janitor");
this.clusterStatusChore = new ClusterStatusChore(this, balancer);
getChoreService().scheduleChore(clusterStatusChore);
this.balancerChore = new BalancerChore(this);
@@ -1105,7 +1110,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
if (!waitForNamespaceOnline()) {
return;
}
- status.setStatus("Starting cluster schema service");
+ startupStatusTask.setStatus("Starting cluster schema service");
try {
initClusterSchemaService();
} catch (IllegalStateException e) {
@@ -1126,7 +1131,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
}
}
- status.markComplete("Initialization successful");
+ startupStatusTask.setStatus("Initialization successful");
LOG.info(String.format("Master has completed initialization %.3fsec",
(EnvironmentEdgeManager.currentTime() - masterActiveTime) / 1000.0f));
this.masterFinishedInitializationTime = EnvironmentEdgeManager.currentTime();
@@ -1167,7 +1172,7 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
}
assignmentManager.checkIfShouldMoveSystemRegionAsync();
- status.setStatus("Starting quota manager");
+ startupStatusTask.setStatus("Starting quota manager");
initQuotaManager();
if (QuotaUtil.isQuotaEnabled(conf)) {
// Create the quota snapshot notifier
@@ -1190,13 +1195,13 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
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");
+ startupStatusTask.setStatus("Checking ZNode ACLs");
zooKeeper.checkAndSetZNodeAcls();
- status.setStatus("Initializing MOB Cleaner");
+ startupStatusTask.setStatus("Initializing MOB Cleaner");
initMobCleaner();
- status.setStatus("Calling postStartMaster coprocessors");
+ startupStatusTask.setStatus("Calling postStartMaster coprocessors");
if (this.cpHost != null) {
// don't let cp initialization errors kill the master
try {
@@ -1212,12 +1217,26 @@ private void finishActiveMasterInitialization(MonitoredTask status) throws IOExc
* After master has started up, lets do balancer post startup initialization. Since this runs
* in activeMasterManager thread, it should be fine.
*/
+ startupStatusTask.setStatus("Calling postMasterStartup Balancer");
long start = EnvironmentEdgeManager.currentTime();
this.balancer.postMasterStartupInitialize();
if (LOG.isDebugEnabled()) {
LOG.debug("Balancer post startup initialization complete, took " + (
(EnvironmentEdgeManager.currentTime() - start) / 1000) + " seconds");
}
+ startupStatusTask.markComplete("Startup Complete");
+ }
+
+ /*
+ Get all the completed/running phases.
+ */
+ public List getStartupPhases() {
+ if (startupStatusTask != null) {
+ return new StartupProgressView(startupStatusTask).getPhaseStatus();
+ } else {
+ // Startup hasn't started yet.
+ return Collections.emptyList();
+ }
}
private void createMissingCFsInMetaDuringUpgrade(
@@ -2271,14 +2290,16 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
Threads.sleep(timeout);
}
}
- MonitoredTask status = TaskMonitor.get().createStatus("Master startup");
- status.setDescription("Master startup");
+ startupStatusTask = TaskMonitor.get().createStatus("Master startup");
+ startupStatusTask.setDescription("Master startup");
+ // Enabling status journal for Startup Progress UI
+ startupStatusTask.enableStatusJournal(false);
try {
- if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, status)) {
- finishActiveMasterInitialization(status);
+ if (activeMasterManager.blockUntilBecomingActiveMaster(timeout, startupStatusTask)) {
+ finishActiveMasterInitialization();
}
} catch (Throwable t) {
- status.setStatus("Failed to become active: " + t.getMessage());
+ startupStatusTask.setStatus("Failed to become active: " + t.getMessage());
LOG.error(HBaseMarkers.FATAL, "Failed to become active master", t);
// HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility
if (t instanceof NoClassDefFoundError && t.getMessage().
@@ -2291,7 +2312,7 @@ private void startActiveMasterManager(int infoPort) throws KeeperException {
abort("Unhandled exception. Starting shutdown.", t);
}
} finally {
- status.cleanup();
+ startupStatusTask.cleanup();
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/startupprogress/Phase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/startupprogress/Phase.java
new file mode 100644
index 000000000000..788f0594dc1e
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/startupprogress/Phase.java
@@ -0,0 +1,73 @@
+/**
+ *
+ * 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.master.startupprogress;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * This stores all the startup progress details that are required on the UI page.
+ */
+@InterfaceAudience.Private
+public class Phase {
+ private String name;
+ private String status;
+ private String startTime;
+ private String endTime;
+ private String elapsedTime;
+
+ public Phase(String name) {
+ this.name = name;
+ }
+
+ public String getName() {
+ return name;
+ }
+
+ public String getStatus() {
+ return status;
+ }
+
+ public void setStatus(String status) {
+ this.status = status;
+ }
+
+ public String getStartTime() {
+ return startTime;
+ }
+
+ public void setStartTime(String startTime) {
+ this.startTime = startTime;
+ }
+
+ public String getEndTime() {
+ return endTime;
+ }
+
+ public void setEndTime(String endTime) {
+ this.endTime = endTime;
+ }
+
+ public String getElapsedTime() {
+ return elapsedTime;
+ }
+
+ public void setElapsedTime(String elapsedTime) {
+ this.elapsedTime = elapsedTime;
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/startupprogress/StartupProgressView.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/startupprogress/StartupProgressView.java
new file mode 100644
index 000000000000..b69ed68b6a87
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/startupprogress/StartupProgressView.java
@@ -0,0 +1,118 @@
+/**
+ *
+ * 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.master.startupprogress;
+
+import static org.apache.hadoop.hbase.util.EnvironmentEdgeManager.currentTime;
+
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.hadoop.hbase.monitoring.MonitoredTask;
+import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/*
+ Creates a view for Startup Progress phases.
+ */
+@InterfaceAudience.Private
+public class StartupProgressView {
+ private MonitoredTask monitoredTask;
+ private DateTimeFormatter formatter =
+ DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSS").withZone(ZoneId.of("UTC"));
+ private static final String COMPLETE = "COMPLETE";
+ private static final String RUNNING = "RUNNING";
+
+ public StartupProgressView(MonitoredTask monitoredTask) {
+ this.monitoredTask = monitoredTask;
+ }
+
+ /**
+ * Return list of phase with starttime, endtime and status
+ * @return list of phases
+ */
+ public List getPhaseStatus() {
+ List phases = new ArrayList<>();
+ // Make a copy of the list to avoid ConcurrentModificationException.
+ List journalEntries =
+ ImmutableList.copyOf(this.monitoredTask.getStatusJournal());
+ for (MonitoredTask.StatusJournalEntry entry: journalEntries) {
+ Phase phase = new Phase(entry.getStatus());
+ phase.setStartTime(formatter.format(Instant.ofEpochMilli(entry.getStartTimeStamp())));
+ if (entry.getEndTimeStamp() == Long.MAX_VALUE) {
+ // This means this task is still RUNNING.
+ phase.setStatus(RUNNING);
+ phase.setEndTime("-");
+ } else {
+ phase.setStatus(COMPLETE);
+ phase.setEndTime(formatter.format(Instant.ofEpochMilli(entry.getEndTimeStamp())));
+ }
+ long elapsedTime = calculateElapsedTime(entry);
+ phase.setElapsedTime(convertElapsedTimeHumanReadableTime(elapsedTime));
+ phases.add(phase);
+ }
+ return phases;
+ }
+
+ /**
+ * Calculate the elapsedTime. If this status is complete then return diff of start and end time.
+ * If the status is still running, then return diff of current time and start time.
+ * @param entry journal entry
+ * @return elapsed time.
+ */
+ private long calculateElapsedTime(MonitoredTask.StatusJournalEntry entry) {
+ if (entry.getEndTimeStamp() != Long.MAX_VALUE) {
+ return entry.getEndTimeStamp() - entry.getStartTimeStamp();
+ } else {
+ return currentTime() - entry.getStartTimeStamp();
+ }
+ }
+
+ /**
+ * Convert the elapsedTime to human readable form (i.e 1 hour 10 mins 20 secs 10 ms)
+ * @param elapsedTime elapsedTime
+ * @return human readable format
+ */
+ private String convertElapsedTimeHumanReadableTime(long elapsedTime) {
+ if (elapsedTime == Long.MAX_VALUE) {
+ return "-";
+ }
+ long hours, minutes, seconds, milliseconds;
+ milliseconds = elapsedTime % 1000;
+ elapsedTime /= 1000;
+ seconds = elapsedTime % 60;
+ elapsedTime /= 60;
+ minutes = elapsedTime % 60;
+ elapsedTime /= 60;
+ hours = elapsedTime % 24;
+ StringBuilder sb = new StringBuilder();
+ if (hours != 0) {
+ sb.append(hours + " hours ");
+ }
+ if (minutes != 0) {
+ sb.append(minutes + " mins ");
+ }
+ if (seconds != 0) {
+ sb.append(seconds + " secs ");
+ }
+ sb.append(milliseconds + " ms ");
+ return sb.toString();
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
index a715549250ef..d74fb1f9aa82 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTask.java
@@ -36,7 +36,9 @@ enum State {
public interface StatusJournalEntry {
String getStatus();
- long getTimeStamp();
+ long getStartTimeStamp();
+ long getEndTimeStamp();
+ void setEndTimeStamp(long endTime);
}
long getStartTime();
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
index 011c19822c94..88e4b6e7c3f2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/monitoring/MonitoredTaskImpl.java
@@ -45,6 +45,7 @@ class MonitoredTaskImpl implements MonitoredTask {
private boolean journalEnabled = false;
private List journal;
+ private StatusJournalEntry lastJournalEntry = null;
private static final Gson GSON = GsonUtil.createGson().create();
@@ -56,12 +57,13 @@ public MonitoredTaskImpl() {
}
private static class StatusJournalEntryImpl implements StatusJournalEntry {
- private long statusTime;
+ private long startTime;
+ private long endTime = Long.MAX_VALUE;
private String status;
- public StatusJournalEntryImpl(String status, long statusTime) {
+ public StatusJournalEntryImpl(String status, long startTime) {
this.status = status;
- this.statusTime = statusTime;
+ this.startTime = startTime;
}
@Override
@@ -70,8 +72,18 @@ public String getStatus() {
}
@Override
- public long getTimeStamp() {
- return statusTime;
+ public long getStartTimeStamp() {
+ return startTime;
+ }
+
+ @Override
+ public long getEndTimeStamp() {
+ return endTime;
+ }
+
+ @Override
+ public void setEndTimeStamp(long endTime) {
+ this.endTime = endTime;
}
@Override
@@ -79,7 +91,10 @@ public String toString() {
StringBuilder sb = new StringBuilder();
sb.append(status);
sb.append(" at ");
- sb.append(statusTime);
+ sb.append(" startTime: " + startTime);
+ if (endTime != Long.MAX_VALUE) {
+ sb.append(" and endTime: " + endTime);
+ }
return sb.toString();
}
}
@@ -140,6 +155,8 @@ public long getCompletionTimestamp() {
public void markComplete(String status) {
setState(State.COMPLETE);
setStatus(status);
+ // We need to set the end time of this status since we don't expect anymore status changes.
+ setLastStatusEndTime();
}
@Override
@@ -158,14 +175,31 @@ public void resume(String msg) {
public void abort(String msg) {
setStatus(msg);
setState(State.ABORTED);
+ // We need to set the end time of this status since we don't expect anymore status changes.
+ setLastStatusEndTime();
}
@Override
public void setStatus(String status) {
+ // Before changing the status, set the last status end time.
+ setLastStatusEndTime();
this.status = status;
- statusTime = EnvironmentEdgeManager.currentTime();
+ startTime = EnvironmentEdgeManager.currentTime();
if (journalEnabled) {
- journal.add(new StatusJournalEntryImpl(this.status, statusTime));
+ StatusJournalEntry entry = new StatusJournalEntryImpl(this.status, startTime);
+ journal.add(entry);
+ lastJournalEntry = entry;
+ }
+ }
+
+ /**
+ * Set the last status end time.
+ */
+ public void setLastStatusEndTime() {
+ if (journalEnabled && status != null) {
+ if (lastJournalEntry != null) {
+ lastJournalEntry.setEndTimeStamp(EnvironmentEdgeManager.currentTime());
+ }
}
}
@@ -281,7 +315,7 @@ public String prettyPrintJournal() {
sb.append(je.toString());
if (i != 0) {
StatusJournalEntry jep = journal.get(i-1);
- long delta = je.getTimeStamp() - jep.getTimeStamp();
+ long delta = je.getStartTimeStamp() - jep.getStartTimeStamp();
if (delta != 0) {
sb.append(" (+" + delta + " ms)");
}
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/header.jsp b/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
index c190337280f5..442e38e71542 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/header.jsp
@@ -75,6 +75,7 @@
<% if (HBaseConfiguration.isShowConfInServlet()) { %>
HBase Configuration
<% } %>
+ Startup Progress
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/startupProgress.jsp b/hbase-server/src/main/resources/hbase-webapps/master/startupProgress.jsp
new file mode 100644
index 000000000000..b79ce6ee5ca4
--- /dev/null
+++ b/hbase-server/src/main/resources/hbase-webapps/master/startupProgress.jsp
@@ -0,0 +1,62 @@
+<%--
+/**
+ * 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.
+ */
+--%>
+<%@ page contentType="text/html;charset=UTF-8"
+%>
+<%@ page import="org.apache.hadoop.hbase.master.HMaster" %>
+<%@ page import="org.apache.hadoop.hbase.master.startupprogress.Phase" %>
+<%
+ HMaster master = (HMaster) getServletContext().getAttribute(HMaster.MASTER);
+ pageContext.setAttribute("pageTitle", "HBase Master: " + master.getServerName());
+%>
+
+
+
+
+
+
+
+
+
+ | Phase |
+ Status |
+ Start Time (UTC) |
+ End Time (UTC) |
+ Elapsed Time |
+
+
+ <% for (Phase phase : master.getStartupPhases()) { %>
+
+ | <%= phase.getName() %> |
+ <%= phase.getStatus() %> |
+ <%= phase.getStartTime() %> |
+ <%= phase.getEndTime() %> |
+ <%= phase.getElapsedTime() %> |
+
+ <% } %>
+
+
+
+
+
+
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AlwaysStandByHMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AlwaysStandByHMaster.java
index 3d36db71242a..f2672230ada1 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AlwaysStandByHMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AlwaysStandByHMaster.java
@@ -55,6 +55,7 @@ private static class AlwaysStandByMasterManager extends ActiveMasterManager {
/**
* An implementation that never transitions to an active master.
*/
+ @Override
boolean blockUntilBecomingActiveMaster(int checkInterval, MonitoredTask startupStatus) {
while (!(master.isAborted() || master.isStopped())) {
startupStatus.setStatus("Forever looping to stay as a standby master.");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
index c9a9fc98069a..79302360c7c0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/monitoring/TestTaskMonitor.java
@@ -220,7 +220,7 @@ public void testStatusJournal() {
task.enableStatusJournal(true);
// check existing status entered in journal
assertEquals("status1", task.getStatusJournal().get(0).getStatus());
- assertTrue(task.getStatusJournal().get(0).getTimeStamp() > 0);
+ assertTrue(task.getStatusJournal().get(0).getStartTimeStamp() > 0);
task.disableStatusJournal();
task.setStatus("status2");
// check status 2 not added since disabled