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 87b8289692c9..7a4a64272434 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,8 @@ AssignmentManager assignmentManager = master.getAssignmentManager(); <%if HBaseConfiguration.isShowConfInServlet()%>
  • HBase Configuration
  • +
  • Startup Progress
  • + diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 9706149e82d5..6ec4ba5338ea 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -163,6 +163,8 @@ import org.apache.hadoop.hbase.master.replication.UpdatePeerConfigProcedure; import org.apache.hadoop.hbase.master.slowlog.SlowLogMasterService; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; +import org.apache.hadoop.hbase.master.startupprogress.Phase; +import org.apache.hadoop.hbase.master.startupprogress.StartupProgressView; import org.apache.hadoop.hbase.master.zksyncer.MasterAddressSyncer; import org.apache.hadoop.hbase.master.zksyncer.MetaLocationSyncer; import org.apache.hadoop.hbase.mob.MobFileCleanerChore; @@ -427,6 +429,7 @@ public class HMaster extends HBaseServerBase implements Maste // Cached clusterId on stand by masters to serve clusterID requests from clients. private final CachedClusterId cachedClusterId; + private MonitoredTask startupStatusTask; /** * Initializes the HMaster. The steps are as follows: @@ -437,7 +440,7 @@ public class HMaster extends HBaseServerBase 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()); +%> + + + + +
    +
    + +
    + + + + + + + + + + + <% for (Phase phase : master.getStartupPhases()) { %> + + + + + + + + <% } %> + +
    PhaseStatusStart Time (UTC)End Time (UTC)Elapsed Time
    <%= 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